HBASE-24640 [branch-2] Purge use of VisibleForTesting (#2696)

Signed-off-by: Nick Dimiduk <ndimiduk@apache.org>
This commit is contained in:
Andrew Purtell 2020-11-25 14:15:32 -08:00 committed by GitHub
parent 3dd425abfa
commit 1b6399a480
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
250 changed files with 462 additions and 1365 deletions

View File

@ -72,13 +72,11 @@ 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.PairOfSameType;
import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
import org.apache.yetus.audience.InterfaceAudience;
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;
/**
* <p>
* Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
@ -148,7 +146,6 @@ public class MetaTableAccessor {
private static final Logger LOG = LoggerFactory.getLogger(MetaTableAccessor.class);
private static final Logger METALOG = LoggerFactory.getLogger("org.apache.hadoop.hbase.META");
@VisibleForTesting
public static final byte[] REPLICATION_PARENT_QUALIFIER = Bytes.toBytes("parent");
private static final byte ESCAPE_BYTE = (byte) 0xFF;
@ -451,7 +448,6 @@ public class MetaTableAccessor {
* true and we'll leave out offlined regions from returned list
* @return List of all user-space regions.
*/
@VisibleForTesting
public static List<RegionInfo> getAllRegions(Connection connection,
boolean excludeOfflinedSplitParents)
throws IOException {
@ -908,7 +904,6 @@ public class MetaTableAccessor {
* @param replicaId the replicaId of the region
* @return a byte[] for server column qualifier
*/
@VisibleForTesting
public static byte[] getServerColumn(int replicaId) {
return replicaId == 0
? HConstants.SERVER_QUALIFIER
@ -921,7 +916,6 @@ public class MetaTableAccessor {
* @param replicaId the replicaId of the region
* @return a byte[] for server start code column qualifier
*/
@VisibleForTesting
public static byte[] getStartCodeColumn(int replicaId) {
return replicaId == 0
? HConstants.STARTCODE_QUALIFIER
@ -934,7 +928,6 @@ public class MetaTableAccessor {
* @param replicaId the replicaId of the region
* @return a byte[] for seqNum column qualifier
*/
@VisibleForTesting
public static byte[] getSeqNumColumn(int replicaId) {
return replicaId == 0
? HConstants.SEQNUM_QUALIFIER
@ -948,7 +941,6 @@ public class MetaTableAccessor {
* @param serverColumn the column qualifier
* @return an int for the replicaId
*/
@VisibleForTesting
static int parseReplicaIdFromServerColumn(byte[] serverColumn) {
String serverStr = Bytes.toString(serverColumn);
@ -1472,7 +1464,6 @@ public class MetaTableAccessor {
* @param regionInfo region information
* @throws IOException if problem connecting or updating meta
*/
@VisibleForTesting
public static void addRegionToMeta(Connection connection, RegionInfo regionInfo)
throws IOException {
addRegionsToMeta(connection, Collections.singletonList(regionInfo), 1);
@ -1699,7 +1690,6 @@ public class MetaTableAccessor {
* merge and split as these want to make atomic mutations across multiple rows.
* @throws IOException even if we encounter a RuntimeException, we'll still wrap it in an IOE.
*/
@VisibleForTesting
static void multiMutate(final Table table, byte[] row, final List<Mutation> mutations)
throws IOException {
debugLogMutations(mutations);
@ -1752,7 +1742,6 @@ public class MetaTableAccessor {
* @param sn Server name
* @param masterSystemTime wall clock time from master if passed in the open region RPC
*/
@VisibleForTesting
public static void updateRegionLocation(Connection connection, RegionInfo regionInfo,
ServerName sn, long openSeqNum, long masterSystemTime) throws IOException {
updateLocation(connection, regionInfo, sn, openSeqNum, masterSystemTime);
@ -1937,7 +1926,6 @@ public class MetaTableAccessor {
}
}
@VisibleForTesting
public static byte[] getParentsBytes(List<RegionInfo> parents) {
ByteArrayOutputStream bos = new ByteArrayOutputStream();
Iterator<RegionInfo> iter = parents.iterator();

View File

@ -30,11 +30,9 @@ import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
import org.apache.hbase.thirdparty.io.netty.util.Timeout;
import org.apache.yetus.audience.InterfaceAudience;
/**
* The implementation of {@link AsyncBufferedMutator}. Simply wrap an {@link AsyncTable}.
@ -60,7 +58,6 @@ class AsyncBufferedMutatorImpl implements AsyncBufferedMutator {
private boolean closed;
@VisibleForTesting
Timeout periodicFlushTask;
AsyncBufferedMutatorImpl(HashedWheelTimer periodicalFlushTimer, AsyncTable<?> table,
@ -83,7 +80,6 @@ class AsyncBufferedMutatorImpl implements AsyncBufferedMutator {
}
// will be overridden in test
@VisibleForTesting
protected void internalFlush() {
if (periodicFlushTask != null) {
periodicFlushTask.cancel();

View File

@ -48,21 +48,18 @@ 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.security.User;
import org.apache.hadoop.hbase.util.ConcurrentMapUtils;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
import org.apache.hadoop.hbase.util.ConcurrentMapUtils;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* The implementation of AsyncConnection.
@ -72,7 +69,6 @@ class AsyncConnectionImpl implements AsyncConnection {
private static final Logger LOG = LoggerFactory.getLogger(AsyncConnectionImpl.class);
@VisibleForTesting
static final HashedWheelTimer RETRY_TIMER = new HashedWheelTimer(
new ThreadFactoryBuilder().setNameFormat("Async-Client-Retry-Timer-pool-%d").setDaemon(true)
.setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build(),
@ -244,7 +240,6 @@ class AsyncConnectionImpl implements AsyncConnection {
}
// ditto
@VisibleForTesting
public NonceGenerator getNonceGenerator() {
return nonceGenerator;
}

View File

@ -59,13 +59,11 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.Scan.ReadType;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hbase.thirdparty.com.google.common.base.Objects;
import org.apache.yetus.audience.InterfaceAudience;
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.Objects;
/**
* The asynchronous locator for regions other than meta.
*/
@ -74,13 +72,11 @@ class AsyncNonMetaRegionLocator {
private static final Logger LOG = LoggerFactory.getLogger(AsyncNonMetaRegionLocator.class);
@VisibleForTesting
static final String MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE =
"hbase.client.meta.max.concurrent.locate.per.table";
private static final int DEFAULT_MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE = 8;
@VisibleForTesting
static String LOCATE_PREFETCH_LIMIT = "hbase.client.locate.prefetch.limit";
private static final int DEFAULT_LOCATE_PREFETCH_LIMIT = 10;
@ -710,7 +706,6 @@ class AsyncNonMetaRegionLocator {
}
// only used for testing whether we have cached the location for a region.
@VisibleForTesting
RegionLocations getRegionLocationInCache(TableName tableName, byte[] row) {
TableCache tableCache = cache.get(tableName);
if (tableCache == null) {

View File

@ -19,9 +19,6 @@
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;
@ -40,14 +37,14 @@ 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;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* This class allows a continuous flow of requests. It's written to be compatible with a
@ -146,7 +143,6 @@ class AsyncProcess {
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 */
@ -156,7 +152,6 @@ class AsyncProcess {
/**
* 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;
@ -422,7 +417,6 @@ class AsyncProcess {
return checkTimeout("rpc timeout", rpcTimeout);
}
@VisibleForTesting
<CResult> AsyncRequestFutureImpl<CResult> createAsyncRequestFuture(
AsyncProcessTask task, List<Action> actions, long nonceGroup) {
return new AsyncRequestFutureImpl<>(task, actions, nonceGroup, this);
@ -456,13 +450,11 @@ class AsyncProcess {
/**
* Create a caller. Isolated to be easily overridden in the tests.
*/
@VisibleForTesting
protected RpcRetryingCaller<AbstractResponse> createCaller(
CancellableRegionServerCallable callable, int rpcTimeout) {
return rpcCallerFactory.<AbstractResponse> 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

View File

@ -30,14 +30,12 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FutureUtils;
import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
import org.apache.hbase.thirdparty.io.netty.util.Timeout;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
import org.apache.hbase.thirdparty.io.netty.util.Timeout;
/**
* The asynchronous region locator.
*/
@ -175,7 +173,6 @@ class AsyncRegionLocator {
nonMetaRegionLocator.clearCache();
}
@VisibleForTesting
AsyncNonMetaRegionLocator getNonMetaRegionLocator() {
return nonMetaRegionLocator;
}

View File

@ -54,8 +54,6 @@ 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
@ -178,13 +176,11 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
* 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<CancellableRegionServerCallable> callsInProgress;
@VisibleForTesting
SingleServerRequestRunnable(
MultiAction multiAction, int numAttempt, ServerName server,
Set<CancellableRegionServerCallable> callsInProgress) {
@ -394,12 +390,10 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
}
}
@VisibleForTesting
protected Set<CancellableRegionServerCallable> getCallsInProgress() {
return callsInProgress;
}
@VisibleForTesting
SingleServerRequestRunnable createSingleServerRequest(MultiAction multiAction, int numAttempt, ServerName server,
Set<CancellableRegionServerCallable> callsInProgress) {
return new SingleServerRequestRunnable(multiAction, numAttempt, server, callsInProgress);
@ -920,13 +914,11 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
}
}
@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) {
@ -1242,7 +1234,6 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
results[index] = result;
}
@VisibleForTesting
long getNumberOfActionsInProgress() {
return actionsInProgress.get();
}

View File

@ -29,8 +29,6 @@ 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 {@link ResultScanner} implementation for {@link AsyncTable}. It will fetch data automatically
* in background and cache it in memory. Typically the {@link #maxCacheSize} will be
@ -177,7 +175,6 @@ class AsyncTableResultScanner implements ResultScanner, AdvancedScanResultConsum
}
// used in tests to test whether the scanner has been suspended
@VisibleForTesting
synchronized boolean isSuspended() {
return resumer != null;
}

View File

@ -40,8 +40,6 @@ import org.apache.yetus.audience.InterfaceStability;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* <p>
* Used to communicate with a single HBase table similar to {@link Table}
@ -94,7 +92,6 @@ public class BufferedMutatorImpl implements BufferedMutator {
private volatile boolean closed = false;
private final AsyncProcess ap;
@VisibleForTesting
BufferedMutatorImpl(ClusterConnection conn, BufferedMutatorParams params, AsyncProcess ap) {
if (conn == null || conn.isClosed()) {
throw new IllegalArgumentException("Connection is null or closed.");
@ -153,12 +150,10 @@ public class BufferedMutatorImpl implements BufferedMutator {
}
}
@VisibleForTesting
ExecutorService getPool() {
return pool;
}
@VisibleForTesting
AsyncProcess getAsyncProcess() {
return ap;
}
@ -203,7 +198,6 @@ public class BufferedMutatorImpl implements BufferedMutator {
doFlush(false);
}
@VisibleForTesting
protected long getExecutedWriteBufferPeriodicFlushes() {
return executedWriteBufferPeriodicFlushes.get();
}
@ -401,7 +395,6 @@ public class BufferedMutatorImpl implements BufferedMutator {
this.operationTimeout.set(operationTimeout);
}
@VisibleForTesting
long getCurrentWriteBufferSize() {
return currentWriteBufferSize.get();
}
@ -410,7 +403,6 @@ public class BufferedMutatorImpl implements BufferedMutator {
* Count the mutations which haven't been processed.
* @return count of undealt mutation
*/
@VisibleForTesting
int size() {
return undealtMutationCount.get();
}
@ -419,17 +411,14 @@ public class BufferedMutatorImpl implements BufferedMutator {
* 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<Row>, Closeable {
private int remainder = undealtMutationCount.getAndSet(0);
private Mutation last = null;

View File

@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* ClientAsyncPrefetchScanner implements async scanner behaviour.
@ -70,7 +69,6 @@ public class ClientAsyncPrefetchScanner extends ClientSimpleScanner {
Threads.setDaemonThreadRunning(new Thread(new PrefetchRunnable()), name + ".asyncPrefetcher");
}
@VisibleForTesting
void setPrefetchListener(Consumer<Boolean> prefetchListener) {
this.prefetchListener = prefetchListener;
}

View File

@ -40,15 +40,12 @@ 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.shaded.protobuf.ProtobufUtil;
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;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
/**
* Implements the scanner interface for the HBase client. If there are multiple regions in a table,
* this scanner will iterate through them all.
@ -186,7 +183,6 @@ public abstract class ClientScanner extends AbstractClientScanner {
return lastNext;
}
@VisibleForTesting
protected long getMaxResultSize() {
return maxScannerResultSize;
}
@ -219,7 +215,6 @@ public abstract class ClientScanner extends AbstractClientScanner {
* 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 {
@ -256,7 +251,6 @@ public abstract class ClientScanner extends AbstractClientScanner {
return true;
}
@VisibleForTesting
boolean isAnyRPCcancelled() {
return callable.isAnyRPCcancelled();
}
@ -323,7 +317,6 @@ public abstract class ClientScanner extends AbstractClientScanner {
return result;
}
@VisibleForTesting
public int getCacheSize() {
return cache != null ? cache.size() : 0;
}
@ -546,7 +539,6 @@ public abstract class ClientScanner extends AbstractClientScanner {
return;
}
@VisibleForTesting
public int getCacheCount() {
return cache != null ? cache.size() : 0;
}

View File

@ -15,8 +15,6 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Configuration parameters for the connection.
* Configuration is a heavy weight registry that does a lot of string operations and regex matching.
@ -124,7 +122,6 @@ public class ConnectionConfiguration {
* This is for internal testing purpose (using the default value).
* In real usage, we should read the configuration from the Configuration object.
*/
@VisibleForTesting
protected ConnectionConfiguration() {
this.writeBufferSize = WRITE_BUFFER_SIZE_DEFAULT;
this.writeBufferPeriodicFlushTimeoutMs = WRITE_BUFFER_PERIODIC_FLUSH_TIMEOUT_MS_DEFAULT;

View File

@ -75,26 +75,6 @@ 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.hbase.util.Threads;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
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;
@ -147,6 +127,23 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.Remov
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
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.hbase.util.Threads;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
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.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Main implementation of {@link Connection} and {@link ClusterConnection} interfaces.
@ -386,7 +383,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
* @param cnm Replaces the nonce generator used, for testing.
* @return old nonce generator.
*/
@VisibleForTesting
static NonceGenerator injectNonceGeneratorForTesting(
ClusterConnection conn, NonceGenerator cnm) {
ConnectionImplementation connImpl = (ConnectionImplementation)conn;
@ -584,7 +580,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
/**
* For tests only.
*/
@VisibleForTesting
RpcClient getRpcClient() {
return rpcClient;
}
@ -2071,7 +2066,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
* 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);
}

View File

@ -49,19 +49,6 @@ import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.net.DNS;
import org.apache.yetus.audience.InterfaceAudience;
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.hbase.thirdparty.io.netty.util.Timer;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
@ -69,6 +56,16 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.net.DNS;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
import org.apache.hbase.thirdparty.io.netty.util.Timer;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Utility used by client connections.
@ -137,7 +134,7 @@ public final class ConnectionUtils {
* if the invocation target is 'this' server; save on network and protobuf invocations.
*/
// TODO This has to still do PB marshalling/unmarshalling stuff. Check how/whether we can avoid.
@VisibleForTesting // Class is visible so can assert we are short-circuiting when expected.
// Class is visible so can assert we are short-circuiting when expected.
public static class ShortCircuitingClusterConnection extends ConnectionImplementation {
private final ServerName serverName;
private final AdminService.BlockingInterface localHostAdmin;
@ -198,7 +195,6 @@ public final class ConnectionUtils {
* 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(ClusterConnection.HBASE_CLIENT_CONNECTION_IMPL, MasterlessConnection.class.getName());
}

View File

@ -92,28 +92,6 @@ 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.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
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;
@ -233,6 +211,25 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetRe
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
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.hbase.thirdparty.com.google.common.base.Preconditions;
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* HBaseAdmin is no longer a client API. It is marked InterfaceAudience.Private indicating that
@ -1664,7 +1661,6 @@ public class HBaseAdmin implements Admin {
* two adjacent regions
* @throws IOException if a remote or network exception occurs
*/
@VisibleForTesting
public void mergeRegionsSync(
final byte[] nameOfRegionA,
final byte[] nameOfRegionB,
@ -1782,12 +1778,10 @@ public class HBaseAdmin implements Admin {
* @param splitPoint key where region splits
* @throws IOException if a remote or network exception occurs
*/
@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

View File

@ -32,21 +32,13 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
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.coprocessor.Batch;
import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
@ -58,6 +50,13 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActi
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.io.InterruptedIOException;
@ -120,7 +119,6 @@ public class HTable implements Table {
private final HRegionLocator locator;
/** The Async process for batch */
@VisibleForTesting
AsyncProcess multiAp;
private final RpcRetryingCallerFactory rpcCallerFactory;
private final RpcControllerFactory rpcControllerFactory;
@ -220,7 +218,6 @@ public class HTable implements Table {
* manipulations.
* @return A Connection instance.
*/
@VisibleForTesting
protected Connection getConnection() {
return this.connection;
}

View File

@ -42,13 +42,11 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.yetus.audience.InterfaceAudience;
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.util.concurrent.ThreadFactoryBuilder;
/**
* HTableMultiplexer provides a thread-safe non blocking PUT API across all the tables. Each put
* will be sharded into different buffer queues based on its destination region server. So each
@ -239,7 +237,7 @@ public class HTableMultiplexer {
return new HTableMultiplexerStatus(serverToFlushWorkerMap);
}
@VisibleForTesting
@InterfaceAudience.Private
LinkedBlockingQueue<PutStatus> getQueue(HRegionLocation addr) {
FlushWorker worker = serverToFlushWorkerMap.get(addr);
if (worker == null) {
@ -257,7 +255,7 @@ public class HTableMultiplexer {
return worker.getQueue();
}
@VisibleForTesting
@InterfaceAudience.Private
ClusterConnection getConnection() {
return this.conn;
}
@ -372,7 +370,7 @@ public class HTableMultiplexer {
}
}
@VisibleForTesting
@InterfaceAudience.Private
static class PutStatus {
final RegionInfo regionInfo;
final Put put;
@ -425,7 +423,7 @@ public class HTableMultiplexer {
}
}
@VisibleForTesting
@InterfaceAudience.Private
static class FlushWorker implements Runnable {
private final HRegionLocation addr;
private final LinkedBlockingQueue<PutStatus> queue;
@ -528,33 +526,33 @@ public class HTableMultiplexer {
return true;
}
@VisibleForTesting
@InterfaceAudience.Private
long getNextDelay(int retryCount) {
return ConnectionUtils.getPauseTime(multiplexer.flushPeriod,
multiplexer.maxAttempts - retryCount - 1);
}
@VisibleForTesting
@InterfaceAudience.Private
AtomicInteger getRetryInQueue() {
return this.retryInQueue;
}
@VisibleForTesting
@InterfaceAudience.Private
int getMaxRetryInQueue() {
return this.maxRetryInQueue;
}
@VisibleForTesting
@InterfaceAudience.Private
AtomicLong getTotalFailedPutCount() {
return this.totalFailedPutCount;
}
@VisibleForTesting
@InterfaceAudience.Private
HTableMultiplexer getMultiplexer() {
return this.multiplexer;
}
@VisibleForTesting
@InterfaceAudience.Private
ScheduledExecutorService getExecutor() {
return this.executor;
}

View File

@ -46,17 +46,6 @@ 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.security.User;
import org.apache.hadoop.hbase.util.DNS.ServerType;
import org.apache.yetus.audience.InterfaceAudience;
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.common.base.Strings;
import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
import org.apache.hbase.thirdparty.com.google.common.net.HostAndPort;
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClientMetaService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterIdRequest;
@ -66,6 +55,14 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetMasters
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetMastersResponseEntry;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetMetaRegionLocationsRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetMetaRegionLocationsResponse;
import org.apache.hadoop.hbase.util.DNS.ServerType;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
import org.apache.hbase.thirdparty.com.google.common.base.Strings;
import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
import org.apache.hbase.thirdparty.com.google.common.net.HostAndPort;
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Master based registry implementation. Makes RPCs to the configured master addresses from config
@ -151,7 +148,6 @@ public class MasterRegistry implements ConnectionRegistry {
* <p/>
* Will be called in {@code HBaseTestingUtility}.
*/
@VisibleForTesting
public static String getMasterAddr(Configuration conf) throws UnknownHostException {
String masterAddrFromConf = conf.get(MASTER_ADDRS_KEY);
if (!Strings.isNullOrEmpty(masterAddrFromConf)) {
@ -331,7 +327,6 @@ public class MasterRegistry implements ConnectionRegistry {
"getMasters()").thenApply(MasterRegistry::transformServerNames);
}
@VisibleForTesting
Set<ServerName> getParsedMasterServers() {
return masterAddr2Stub.keySet();
}

View File

@ -26,7 +26,6 @@ import com.codahale.metrics.JmxReporter;
import com.codahale.metrics.MetricRegistry;
import com.codahale.metrics.RatioGauge;
import com.codahale.metrics.Timer;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
@ -34,14 +33,15 @@ import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.function.Supplier;
import org.apache.hadoop.hbase.ServerName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
import org.apache.yetus.audience.InterfaceAudience;
/**
* This class is for maintaining the various connection statistics and publishing them through
@ -126,12 +126,11 @@ public class MetricsConnection implements StatisticTrackable {
}
}
@VisibleForTesting
protected static final class CallTracker {
private final String name;
@VisibleForTesting final Timer callTimer;
@VisibleForTesting final Histogram reqHist;
@VisibleForTesting final Histogram respHist;
final Timer callTimer;
final Histogram reqHist;
final Histogram respHist;
private CallTracker(MetricRegistry registry, String name, String subName, String scope) {
StringBuilder sb = new StringBuilder(CLIENT_SVC).append("_").append(name);
@ -182,7 +181,6 @@ public class MetricsConnection implements StatisticTrackable {
}
}
@VisibleForTesting
protected static class RunnerStats {
final Counter normalRunners;
final Counter delayRunners;
@ -210,7 +208,6 @@ public class MetricsConnection implements StatisticTrackable {
}
}
@VisibleForTesting
protected ConcurrentHashMap<ServerName, ConcurrentMap<byte[], RegionStats>> serverStats
= new ConcurrentHashMap<>();
@ -275,36 +272,36 @@ public class MetricsConnection implements StatisticTrackable {
// static metrics
@VisibleForTesting protected final Counter metaCacheHits;
@VisibleForTesting protected final Counter metaCacheMisses;
@VisibleForTesting protected final CallTracker getTracker;
@VisibleForTesting protected final CallTracker scanTracker;
@VisibleForTesting protected final CallTracker appendTracker;
@VisibleForTesting protected final CallTracker deleteTracker;
@VisibleForTesting protected final CallTracker incrementTracker;
@VisibleForTesting protected final CallTracker putTracker;
@VisibleForTesting protected final CallTracker multiTracker;
@VisibleForTesting protected final RunnerStats runnerStats;
@VisibleForTesting protected final Counter metaCacheNumClearServer;
@VisibleForTesting protected final Counter metaCacheNumClearRegion;
@VisibleForTesting protected final Counter hedgedReadOps;
@VisibleForTesting protected final Counter hedgedReadWin;
@VisibleForTesting protected final Histogram concurrentCallsPerServerHist;
@VisibleForTesting protected final Histogram numActionsPerServerHist;
protected final Counter metaCacheHits;
protected final Counter metaCacheMisses;
protected final CallTracker getTracker;
protected final CallTracker scanTracker;
protected final CallTracker appendTracker;
protected final CallTracker deleteTracker;
protected final CallTracker incrementTracker;
protected final CallTracker putTracker;
protected final CallTracker multiTracker;
protected final RunnerStats runnerStats;
protected final Counter metaCacheNumClearServer;
protected final Counter metaCacheNumClearRegion;
protected final Counter hedgedReadOps;
protected final Counter hedgedReadWin;
protected final Histogram concurrentCallsPerServerHist;
protected final Histogram numActionsPerServerHist;
// dynamic metrics
// These maps are used to cache references to the metric instances that are managed by the
// registry. I don't think their use perfectly removes redundant allocations, but it's
// a big improvement over calling registry.newMetric each time.
@VisibleForTesting protected final ConcurrentMap<String, Timer> rpcTimers =
protected final ConcurrentMap<String, Timer> rpcTimers =
new ConcurrentHashMap<>(CAPACITY, LOAD_FACTOR, CONCURRENCY_LEVEL);
@VisibleForTesting protected final ConcurrentMap<String, Histogram> rpcHistograms =
protected final ConcurrentMap<String, Histogram> rpcHistograms =
new ConcurrentHashMap<>(CAPACITY * 2 /* tracking both request and response sizes */,
LOAD_FACTOR, CONCURRENCY_LEVEL);
private final ConcurrentMap<String, Counter> cacheDroppingExceptions =
new ConcurrentHashMap<>(CAPACITY, LOAD_FACTOR, CONCURRENCY_LEVEL);
@VisibleForTesting protected final ConcurrentMap<String, Counter> rpcCounters =
protected final ConcurrentMap<String, Counter> rpcCounters =
new ConcurrentHashMap<>(CAPACITY, LOAD_FACTOR, CONCURRENCY_LEVEL);
MetricsConnection(String scope, Supplier<ThreadPoolExecutor> batchPool,
@ -358,17 +355,14 @@ public class MetricsConnection implements StatisticTrackable {
this.reporter.start();
}
@VisibleForTesting
final String getExecutorPoolName() {
return name(getClass(), "executorPoolActiveThreads", scope);
}
@VisibleForTesting
final String getMetaPoolName() {
return name(getClass(), "metaPoolActiveThreads", scope);
}
@VisibleForTesting
MetricRegistry getMetricRegistry() {
return registry;
}

View File

@ -39,8 +39,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationPr
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction;
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Callable that handles the <code>multi</code> method call going against a single
* regionserver; i.e. A RegionServerCallable for the multi call (It is NOT a
@ -150,7 +148,6 @@ class MultiServerCallable extends CancellableRegionServerCallable<MultiResponse>
setStub(getConnection().getClient(this.location.getServerName()));
}
@VisibleForTesting
ServerName getServerName() {
return location.getServerName();
}

View File

@ -19,8 +19,6 @@ package org.apache.hadoop.hbase.client;
import static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsent;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import java.io.IOException;
import java.util.Map.Entry;
import java.util.concurrent.ConcurrentHashMap;
@ -30,14 +28,14 @@ import org.apache.commons.lang3.mutable.MutableBoolean;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ServerName;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil;
import org.apache.hadoop.hbase.exceptions.PreemptiveFastFailException;
import org.apache.hadoop.hbase.ipc.CallTimeoutException;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* The concrete {@link RetryingCallerInterceptor} class that implements the preemptive fast fail
@ -149,7 +147,6 @@ class PreemptiveFastFailInterceptor extends RetryingCallerInterceptor {
* - the throwable to be handled.
* @throws PreemptiveFastFailException
*/
@VisibleForTesting
protected void handleFailureToServer(ServerName serverName, Throwable t) {
if (serverName == null || t == null) {
return;

View File

@ -89,23 +89,6 @@ 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.RestoreSnapshotException;
import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
import org.apache.yetus.audience.InterfaceAudience;
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.io.netty.util.HashedWheelTimer;
import org.apache.hbase.thirdparty.io.netty.util.Timeout;
import org.apache.hbase.thirdparty.io.netty.util.TimerTask;
import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
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;
@ -299,6 +282,21 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.Remov
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
import org.apache.hbase.thirdparty.io.netty.util.Timeout;
import org.apache.hbase.thirdparty.io.netty.util.TimerTask;
import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* The implementation of AsyncAdmin.
@ -2356,7 +2354,6 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
* @param regionNameOrEncodedRegionName region name or encoded region name
* @return region location, wrapped by a {@link CompletableFuture}
*/
@VisibleForTesting
CompletableFuture<HRegionLocation> getRegionLocation(byte[] regionNameOrEncodedRegionName) {
if (regionNameOrEncodedRegionName == null) {
return failedFuture(new IllegalArgumentException("Passed region name can't be null"));

View File

@ -18,8 +18,6 @@
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.HashSet;
@ -37,11 +35,11 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ScannerCallable.MoreResults;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.client.ScannerCallable.MoreResults;
import org.apache.hadoop.hbase.util.Pair;
/**
* This class has the logic for handling scanners for regions with and without replicas.
@ -364,7 +362,6 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
callable.getScan().withStartRow(this.lastResult.getRow(), this.lastResult.mayHaveMoreCellsInRow());
}
@VisibleForTesting
boolean isAnyRPCcancelled() {
return someRPCcancelled;
}

View File

@ -18,7 +18,8 @@
*/
package org.apache.hadoop.hbase.client;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsent;
import java.io.InterruptedIOException;
import java.util.ArrayList;
import java.util.Collection;
@ -40,14 +41,13 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdge;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
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.util.Bytes;
import static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsent;
import org.apache.hadoop.hbase.util.EnvironmentEdge;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
/**
* Holds back the requests if they reach any thresholds.
@ -64,7 +64,6 @@ class SimpleRequestController implements RequestController {
/**
* Default value of {@link #HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE}.
*/
@VisibleForTesting
static final long DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE = 4194304;
/**
@ -74,7 +73,6 @@ class SimpleRequestController implements RequestController {
/**
* Default value of {@link #HBASE_CLIENT_MAX_PERREQUEST_ROWS}.
*/
@VisibleForTesting
static final long DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_ROWS = 2048;
/**
@ -84,14 +82,10 @@ class SimpleRequestController implements RequestController {
/**
* Default value of {@link #HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE}.
*/
@VisibleForTesting
static final long DEFAULT_HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE = DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE;
@VisibleForTesting
final AtomicLong tasksInProgress = new AtomicLong(0);
@VisibleForTesting
final ConcurrentMap<byte[], AtomicInteger> taskCounterPerRegion
= new ConcurrentSkipListMap<>(Bytes.BYTES_COMPARATOR);
@VisibleForTesting
final ConcurrentMap<ServerName, AtomicInteger> taskCounterPerServer = new ConcurrentHashMap<>();
/**
* The number of tasks simultaneously executed on the cluster.
@ -113,13 +107,11 @@ class SimpleRequestController implements RequestController {
* don't start a set of operations on a region before the previous one is
* done. As well, this limits the pressure we put on the region server.
*/
@VisibleForTesting
final int maxConcurrentTasksPerRegion;
/**
* The number of task simultaneously executed on a single region server.
*/
@VisibleForTesting
final int maxConcurrentTasksPerServer;
private final int thresholdToLogUndoneTaskDetails;
public static final String THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS =
@ -172,7 +164,6 @@ class SimpleRequestController implements RequestController {
return value;
}
@VisibleForTesting
static Checker newChecker(List<RowChecker> checkers) {
return new Checker() {
private boolean isEnd = false;
@ -332,7 +323,6 @@ class SimpleRequestController implements RequestController {
* limit the heapsize of total submitted data. Reduce the limit of heapsize
* for submitting quickly if there is no running task.
*/
@VisibleForTesting
static class SubmittedSizeChecker implements RowChecker {
private final long maxHeapSizeSubmit;
@ -366,7 +356,6 @@ class SimpleRequestController implements RequestController {
/**
* limit the max number of tasks in an AsyncProcess.
*/
@VisibleForTesting
static class TaskCountChecker implements RowChecker {
private static final long MAX_WAITING_TIME = 1000; //ms
@ -477,7 +466,6 @@ class SimpleRequestController implements RequestController {
/**
* limit the number of rows for each request.
*/
@VisibleForTesting
static class RequestRowsChecker implements RowChecker {
private final long maxRowsPerRequest;
@ -516,7 +504,6 @@ class SimpleRequestController implements RequestController {
/**
* limit the heap size for each request.
*/
@VisibleForTesting
static class RequestHeapSizeChecker implements RowChecker {
private final long maxHeapSizePerRequest;
@ -556,7 +543,6 @@ class SimpleRequestController implements RequestController {
/**
* Provide a way to control the flow of rows iteration.
*/
@VisibleForTesting
interface RowChecker {
ReturnCode canTakeOperation(HRegionLocation loc, long heapSizeOfRow);

View File

@ -24,6 +24,7 @@ import static org.apache.hadoop.hbase.client.RegionReplicaUtil.getRegionInfoForR
import static org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.lengthOfPBMagic;
import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
import static org.apache.hadoop.hbase.zookeeper.ZKMetadata.removeMetaData;
import java.io.IOException;
import java.util.Collection;
import java.util.List;
@ -42,10 +43,12 @@ import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.ReadOnlyZKClient;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
@ -99,7 +102,6 @@ class ZKConnectionRegistry implements ConnectionRegistry {
return getAndConvert(znodePaths.clusterIdZNode, ZKConnectionRegistry::getClusterId);
}
@VisibleForTesting
ReadOnlyZKClient getZKClient() {
return zk;
}

View File

@ -37,13 +37,11 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.replication.ReplicationException;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
import org.apache.yetus.audience.InterfaceAudience;
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.collect.Lists;
/**
* <p>
* This class provides the administrative interface to HBase cluster
@ -388,7 +386,7 @@ public class ReplicationAdmin implements Closeable {
/**
* @deprecated use {@link org.apache.hadoop.hbase.client.Admin#listReplicationPeers()} instead
*/
@VisibleForTesting
@InterfaceAudience.Private
@Deprecated
List<ReplicationPeerDescription> listReplicationPeers() throws IOException {
return admin.listReplicationPeers();

View File

@ -28,7 +28,6 @@ import java.net.SocketTimeoutException;
import java.nio.channels.ClosedChannelException;
import java.util.Set;
import java.util.concurrent.TimeoutException;
import org.apache.hadoop.hbase.CallDroppedException;
import org.apache.hadoop.hbase.CallQueueTooBigException;
import org.apache.hadoop.hbase.DoNotRetryIOException;
@ -36,16 +35,14 @@ import org.apache.hadoop.hbase.MultiActionResultTooLarge;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.RegionTooBusyException;
import org.apache.hadoop.hbase.RetryImmediatelyException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet;
import org.apache.hadoop.hbase.ipc.CallTimeoutException;
import org.apache.hadoop.hbase.ipc.FailedServerException;
import org.apache.hadoop.hbase.quotas.RpcThrottlingException;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet;
@InterfaceAudience.Private
@InterfaceStability.Evolving
@ -150,7 +147,6 @@ public final class ClientExceptionsUtil {
* For test only. Usually you should use the {@link #isConnectionException(Throwable)} method
* below.
*/
@VisibleForTesting
public static Set<Class<? extends Throwable>> getConnectionExceptionTypes() {
return CONNECTION_EXCEPTION_TYPES;
}

View File

@ -37,8 +37,6 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.UnsafeAvailChecker;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* This is optimized version of a standard FuzzyRowFilter Filters data based on fuzzy row key.
* Performs fast-forwards during scanning. It takes pairs (row key, fuzzy info) to match row keys.
@ -100,7 +98,6 @@ public class FuzzyRowFilter extends FilterBase {
this.tracker = new RowTracker();
}
private void preprocessSearchKey(Pair<byte[], byte[]> p) {
if (!UNSAFE_UNALIGNED) {
// do nothing
@ -323,12 +320,12 @@ public class FuzzyRowFilter extends FilterBase {
NO_NEXT
}
@VisibleForTesting
@InterfaceAudience.Private
static SatisfiesCode satisfies(byte[] row, byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) {
return satisfies(false, row, 0, row.length, fuzzyKeyBytes, fuzzyKeyMeta);
}
@VisibleForTesting
@InterfaceAudience.Private
static SatisfiesCode satisfies(boolean reverse, byte[] row, byte[] fuzzyKeyBytes,
byte[] fuzzyKeyMeta) {
return satisfies(reverse, row, 0, row.length, fuzzyKeyBytes, fuzzyKeyMeta);
@ -444,12 +441,12 @@ public class FuzzyRowFilter extends FilterBase {
return SatisfiesCode.YES;
}
@VisibleForTesting
@InterfaceAudience.Private
static byte[] getNextForFuzzyRule(byte[] row, byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) {
return getNextForFuzzyRule(false, row, 0, row.length, fuzzyKeyBytes, fuzzyKeyMeta);
}
@VisibleForTesting
@InterfaceAudience.Private
static byte[] getNextForFuzzyRule(boolean reverse, byte[] row, byte[] fuzzyKeyBytes,
byte[] fuzzyKeyMeta) {
return getNextForFuzzyRule(reverse, row, 0, row.length, fuzzyKeyBytes, fuzzyKeyMeta);
@ -536,7 +533,7 @@ public class FuzzyRowFilter extends FilterBase {
* @return greater byte array than given (row) which satisfies the fuzzy rule if it exists, null
* otherwise
*/
@VisibleForTesting
@InterfaceAudience.Private
static byte[] getNextForFuzzyRule(boolean reverse, byte[] row, int offset, int length,
byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) {
// To find out the next "smallest" byte array that satisfies fuzzy rule and "greater" than

View File

@ -44,12 +44,11 @@ import org.apache.hadoop.hbase.util.PoolMap;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.yetus.audience.InterfaceAudience;
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.util.concurrent.ThreadFactoryBuilder;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder;
import org.apache.hbase.thirdparty.com.google.common.cache.CacheLoader;
@ -216,7 +215,6 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
}
}
@VisibleForTesting
public static String getDefaultCodec(final Configuration c) {
// If "hbase.client.default.rpc.codec" is empty string -- you can't set it to null because
// Configuration will complain -- then no default codec (and we'll pb everything). Else
@ -248,7 +246,6 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
}
// for writing tests that want to throw exception when connecting.
@VisibleForTesting
boolean isTcpNoDelay() {
return tcpNoDelay;
}
@ -562,7 +559,6 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
/**
* Blocking rpc channel that goes via hbase rpc.
*/
@VisibleForTesting
public static class BlockingRpcChannelImplementation extends AbstractRpcChannel
implements BlockingRpcChannel {

View File

@ -25,7 +25,6 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.client.MetricsConnection;
import org.apache.hadoop.net.NetUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Does RPC against a cluster. Manages connections per regionserver in the cluster.
@ -41,7 +40,6 @@ public class BlockingRpcClient extends AbstractRpcClient<BlockingRpcConnection>
* Used in test only. Construct an IPC client for the cluster {@code clusterId} with the default
* SocketFactory
*/
@VisibleForTesting
BlockingRpcClient(Configuration conf) {
this(conf, HConstants.CLUSTER_ID_DEFAULT, null, null);
}

View File

@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.yetus.audience.InterfaceAudience;
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.CodedOutputStream;
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
@ -248,7 +247,6 @@ class IPCUtil {
}
};
@VisibleForTesting
static final int MAX_DEPTH = 4;
static void execute(EventLoop eventLoop, Runnable action) {

View File

@ -17,15 +17,13 @@
*/
package org.apache.hadoop.hbase.ipc;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
import java.net.SocketAddress;
import org.apache.hadoop.conf.Configuration;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.MetricsConnection;
import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
/**
* Factory to create a {@link org.apache.hadoop.hbase.ipc.RpcClient}
@ -46,7 +44,6 @@ public final class RpcClientFactory {
}
/** Helper method for tests only. Creates an {@code RpcClient} without metrics. */
@VisibleForTesting
public static RpcClient createClient(Configuration conf, String clusterId) {
return createClient(conf, clusterId, null);
}

View File

@ -23,7 +23,6 @@ import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
@ -188,7 +187,6 @@ public class RegionState {
// The duration of region in transition
private long ritDuration;
@VisibleForTesting
public static RegionState createForTesting(RegionInfo region, State state) {
return new RegionState(region, state, System.currentTimeMillis(), null);
}

View File

@ -19,12 +19,11 @@ package org.apache.hadoop.hbase.quotas;
import java.io.IOException;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
@ -52,7 +51,6 @@ public class ThrottleSettings extends QuotaSettings {
/**
* Returns a copy of the internal state of <code>this</code>
*/
@VisibleForTesting
QuotaProtos.ThrottleRequest getProto() {
return proto.toBuilder().build();
}

View File

@ -113,31 +113,6 @@ import org.apache.hadoop.hbase.replication.ReplicationLoadSink;
import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
import org.apache.hadoop.hbase.security.visibility.Authorizations;
import org.apache.hadoop.hbase.security.visibility.CellVisibility;
import org.apache.hadoop.hbase.util.Addressing;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.DynamicClassLoader;
import org.apache.hadoop.hbase.util.ExceptionUtil;
import org.apache.hadoop.hbase.util.Methods;
import org.apache.hadoop.hbase.util.VersionInfo;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.io.ByteStreams;
import org.apache.hbase.thirdparty.com.google.gson.JsonArray;
import org.apache.hbase.thirdparty.com.google.gson.JsonElement;
import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
import org.apache.hbase.thirdparty.com.google.protobuf.CodedInputStream;
import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
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.ClearSlowLogResponses;
@ -209,6 +184,28 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDe
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor.EventType;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.util.Addressing;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.DynamicClassLoader;
import org.apache.hadoop.hbase.util.ExceptionUtil;
import org.apache.hadoop.hbase.util.Methods;
import org.apache.hadoop.hbase.util.VersionInfo;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hbase.thirdparty.com.google.common.io.ByteStreams;
import org.apache.hbase.thirdparty.com.google.gson.JsonArray;
import org.apache.hbase.thirdparty.com.google.gson.JsonElement;
import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
import org.apache.hbase.thirdparty.com.google.protobuf.CodedInputStream;
import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Protobufs utility.
@ -288,7 +285,6 @@ public final class ProtobufUtil {
}
}
@VisibleForTesting
public static boolean isClassLoaderLoaded() {
return classLoaderLoaded;
}

View File

@ -42,8 +42,6 @@ import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* A very simple read only zookeeper implementation without watcher support.
*/
@ -117,7 +115,6 @@ public final class ReadOnlyZKClient implements Closeable {
private final AtomicBoolean closed = new AtomicBoolean(false);
@VisibleForTesting
ZooKeeper zookeeper;
private int pendingRequests = 0;
@ -365,7 +362,6 @@ public final class ReadOnlyZKClient implements Closeable {
}
}
@VisibleForTesting
public String getConnectString() {
return connectString;
}

View File

@ -25,8 +25,6 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* This Cell is an implementation of {@link ByteBufferExtendedCell} where the data resides in
* off heap/ on heap ByteBuffer
@ -55,12 +53,10 @@ public class ByteBufferKeyValue extends ByteBufferExtendedCell {
this.length = length;
}
@VisibleForTesting
public ByteBuffer getBuffer() {
return this.buf;
}
@VisibleForTesting
public int getOffset() {
return this.offset;
}

View File

@ -43,8 +43,6 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceAudience.Private;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Utility methods helpful for slinging {@link Cell} instances. Some methods below are for internal
* use only and are marked InterfaceAudience.Private at the method level. Note that all such methods
@ -1672,7 +1670,7 @@ public final class CellUtil {
* than right equal to 0 if left is equal to right
* @deprecated As of HBase-2.0. Will be removed in HBase-3.0
*/
@VisibleForTesting
@InterfaceAudience.Private
@Deprecated
public static final int compare(CellComparator comparator, Cell left, byte[] key, int offset,
int length) {

View File

@ -32,8 +32,6 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* ChoreService is a service that can be used to schedule instances of {@link ScheduledChore} to run
* periodically while sharing threads. The ChoreService is backed by a
@ -95,7 +93,6 @@ public class ChoreService implements ChoreServicer {
* spawned by this service
*/
@InterfaceAudience.Private
@VisibleForTesting
public ChoreService(final String coreThreadPoolPrefix) {
this(coreThreadPoolPrefix, MIN_CORE_POOL_SIZE, false);
}

View File

@ -40,8 +40,6 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* An HBase Key/Value. This is the fundamental HBase Type.
* <p>
@ -1232,7 +1230,6 @@ public class KeyValue implements ExtendedCell, Cloneable {
* and that we need access to the backing array to do some test case related assertions.
* @return The byte array backing this KeyValue.
*/
@VisibleForTesting
public byte [] getBuffer() {
return this.bytes;
}

View File

@ -40,8 +40,6 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Utility methods helpful slinging {@link Cell} instances. It has more powerful and
* rich set of APIs than those in {@link CellUtil} for internal usage.
@ -2608,7 +2606,6 @@ public final class PrivateCellUtil {
* @return an int greater than 0 if left is greater than right lesser than 0 if left is lesser
* than right equal to 0 if left is equal to right
*/
@VisibleForTesting
public static final int compare(CellComparator comparator, Cell left, byte[] key, int offset,
int length) {
// row

View File

@ -24,8 +24,6 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* ScheduledChore is a task performed on a period in hbase. ScheduledChores become active once
* scheduled with a {@link ChoreService} via {@link ChoreService#scheduleChore(ScheduledChore)}. The
@ -116,7 +114,6 @@ public abstract class ScheduledChore implements Runnable {
* This constructor is for test only. It allows us to create an object and to call chore() on it.
*/
@InterfaceAudience.Private
@VisibleForTesting
protected ScheduledChore() {
this("TestChore", null, 0, DEFAULT_INITIAL_DELAY, DEFAULT_TIME_UNIT);
}
@ -313,17 +310,17 @@ public abstract class ScheduledChore implements Runnable {
return initialChoreComplete;
}
@VisibleForTesting
@InterfaceAudience.Private
synchronized ChoreServicer getChoreServicer() {
return choreServicer;
}
@VisibleForTesting
@InterfaceAudience.Private
synchronized long getTimeOfLastRun() {
return timeOfLastRun;
}
@VisibleForTesting
@InterfaceAudience.Private
synchronized long getTimeOfThisRun() {
return timeOfThisRun;
}
@ -336,7 +333,6 @@ public abstract class ScheduledChore implements Runnable {
}
@InterfaceAudience.Private
@VisibleForTesting
public synchronized void choreForTesting() {
chore();
}

View File

@ -25,7 +25,6 @@ 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;
/**
* Maintains the set of all the classes which would like to get notified

View File

@ -25,7 +25,6 @@ import java.util.Queue;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.LongAdder;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.nio.ByteBuff;
@ -33,12 +32,10 @@ import org.apache.hadoop.hbase.nio.SingleByteBuff;
import org.apache.yetus.audience.InterfaceAudience;
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.collect.Sets;
import sun.nio.ch.DirectBuffer;
import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
/**
* ByteBuffAllocator is used for allocating/freeing the ByteBuffers from/to NIO ByteBuffer pool, and
* it provide high-level interfaces for upstream. when allocating desired memory size, it will
@ -212,7 +209,6 @@ public class ByteBuffAllocator {
return new ByteBuffAllocator(false, 0, DEFAULT_BUFFER_SIZE, Integer.MAX_VALUE);
}
@VisibleForTesting
ByteBuffAllocator(boolean reservoirEnabled, int maxBufCount, int bufSize,
int minSizeForReservoirUse) {
this.reservoirEnabled = reservoirEnabled;
@ -245,7 +241,6 @@ public class ByteBuffAllocator {
* The {@link ConcurrentLinkedQueue#size()} is O(N) complexity and time-consuming, so DO NOT use
* the method except in UT.
*/
@VisibleForTesting
public int getFreeBufferCount() {
return this.buffers.size();
}
@ -352,7 +347,6 @@ public class ByteBuffAllocator {
/**
* Free all direct buffers if allocated, mainly used for testing.
*/
@VisibleForTesting
public void clean() {
while (!buffers.isEmpty()) {
ByteBuffer b = buffers.poll();

View File

@ -23,9 +23,7 @@ import java.io.OutputStream;
import java.security.GeneralSecurityException;
import java.security.Key;
import java.security.SecureRandom;
import javax.crypto.spec.SecretKeySpec;
import org.apache.hadoop.hbase.io.crypto.Cipher;
import org.apache.hadoop.hbase.io.crypto.CipherProvider;
import org.apache.hadoop.hbase.io.crypto.Context;
@ -36,7 +34,6 @@ 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;
/**
@ -149,7 +146,6 @@ public class AES extends Cipher {
return d.createDecryptionStream(in);
}
@VisibleForTesting
SecureRandom getRNG() {
return rng;
}

View File

@ -24,9 +24,7 @@ import java.security.GeneralSecurityException;
import java.security.Key;
import java.security.SecureRandom;
import java.util.Properties;
import javax.crypto.spec.SecretKeySpec;
import org.apache.commons.crypto.cipher.CryptoCipherFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.io.crypto.Cipher;
@ -39,7 +37,6 @@ 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;
@InterfaceAudience.Private
@ -159,7 +156,6 @@ public class CommonsCryptoAES extends Cipher {
return decryptor.createDecryptionStream(in);
}
@VisibleForTesting
SecureRandom getRNG() {
return rng;
}

View File

@ -26,7 +26,6 @@ import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import org.apache.commons.lang3.NotImplementedException;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HConstants;
@ -39,7 +38,6 @@ import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.compress.Compressor;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
/**
@ -48,7 +46,6 @@ import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
* This is used only in testing.
*/
@InterfaceAudience.Private
@VisibleForTesting
public class EncodedDataBlock {
private byte[] rawKVs;
private ByteBuffer rawBuffer;

View File

@ -27,8 +27,6 @@ import org.apache.hadoop.fs.PositionedReadable;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* The ThrottleInputStream provides bandwidth throttling on a specified
* InputStream. It is implemented as a wrapper on top of another InputStream
@ -126,7 +124,6 @@ public class ThrottledInputStream extends InputStream {
EnvironmentEdgeManager.currentTime() - startTime);
}
@VisibleForTesting
static long calSleepTimeMs(long bytesRead, long maxBytesPerSec, long elapsed) {
assert elapsed > 0 : "The elapsed time should be greater than zero";
if (bytesRead <= 0 || maxBytesPerSec <= 0) {

View File

@ -23,7 +23,6 @@ import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.hbase.BaseConfigurable;
@ -32,7 +31,6 @@ import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder;
import org.apache.hbase.thirdparty.com.google.common.cache.CacheLoader;
import org.apache.hbase.thirdparty.com.google.common.cache.LoadingCache;
@ -57,7 +55,6 @@ public class UserProvider extends BaseConfigurable {
static Groups groups = Groups.getUserToGroupsMappingService();
@VisibleForTesting
public static Groups getGroups() {
return groups;
}

View File

@ -34,8 +34,6 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* This class manages an array of ByteBuffers with a default size 4MB. These buffers are sequential
* and could be considered as a large buffer.It supports reading/writing data from this large buffer
@ -61,7 +59,6 @@ public class ByteBufferArray {
Runtime.getRuntime().availableProcessors(), capacity, allocator);
}
@VisibleForTesting
ByteBufferArray(int bufferSize, int bufferCount, int threadCount, long capacity,
ByteBufferAllocator alloc) throws IOException {
this.bufferSize = bufferSize;
@ -107,7 +104,6 @@ public class ByteBufferArray {
}
}
@VisibleForTesting
static int getBufferSize(long capacity) {
int bufferSize = DEFAULT_BUFFER_SIZE;
if (bufferSize > (capacity / 16)) {

View File

@ -34,9 +34,8 @@ import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.WritableUtils;
import org.apache.yetus.audience.InterfaceAudience;
import sun.nio.ch.DirectBuffer;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import sun.nio.ch.DirectBuffer;
/**
* Utility functions for working with byte buffers, such as reading/writing
@ -51,7 +50,7 @@ public final class ByteBufferUtils {
public final static int VALUE_MASK = 0x7f;
public final static int NEXT_BIT_SHIFT = 7;
public final static int NEXT_BIT_MASK = 1 << 7;
@VisibleForTesting
@InterfaceAudience.Private
final static boolean UNSAFE_AVAIL = UnsafeAvailChecker.isAvailable();
public final static boolean UNSAFE_UNALIGNED = UnsafeAvailChecker.unaligned();

View File

@ -21,6 +21,7 @@ import static org.apache.hbase.thirdparty.com.google.common.base.Preconditions.c
import static org.apache.hbase.thirdparty.com.google.common.base.Preconditions.checkNotNull;
import static org.apache.hbase.thirdparty.com.google.common.base.Preconditions.checkPositionIndex;
import com.google.protobuf.ByteString;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
@ -37,7 +38,6 @@ import java.util.Collections;
import java.util.Comparator;
import java.util.Iterator;
import java.util.List;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.KeyValue;
@ -47,14 +47,10 @@ import org.apache.hadoop.io.WritableUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
import com.google.protobuf.ByteString;
import sun.misc.Unsafe;
import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
/**
* Utility class that handles byte arrays, conversions to/from other types,
* comparisons, hash code generation, manufacturing keys for HashMaps or
@ -130,7 +126,7 @@ public class Bytes implements Comparable<Bytes> {
// SizeOf which uses java.lang.instrument says 24 bytes. (3 longs?)
public static final int ESTIMATED_HEAP_TAX = 16;
@VisibleForTesting
@InterfaceAudience.Private
static final boolean UNSAFE_UNALIGNED = UnsafeAvailChecker.unaligned();
/**
@ -1397,7 +1393,7 @@ public class Bytes implements Comparable<Bytes> {
}
@VisibleForTesting
@InterfaceAudience.Private
static Comparer<byte[]> lexicographicalComparerJavaImpl() {
return LexicographicalComparerHolder.PureJavaComparer.INSTANCE;
}
@ -1547,7 +1543,7 @@ public class Bytes implements Comparable<Bytes> {
* <p>Uses reflection to gracefully fall back to the Java implementation if
* {@code Unsafe} isn't available.
*/
@VisibleForTesting
@InterfaceAudience.Private
static class LexicographicalComparerHolder {
static final String UNSAFE_COMPARER_NAME =
LexicographicalComparerHolder.class.getName() + "$UnsafeComparer";
@ -1597,7 +1593,7 @@ public class Bytes implements Comparable<Bytes> {
}
}
@VisibleForTesting
@InterfaceAudience.Private
enum UnsafeComparer implements Comparer<byte[]> {
INSTANCE;

View File

@ -28,8 +28,6 @@ import java.util.concurrent.ConcurrentSkipListMap;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Class for determining the "size" of a class, an attempt to calculate the
@ -236,7 +234,6 @@ public class ClassSize {
private static final MemoryLayout memoryLayout = getMemoryLayout();
private static final boolean USE_UNSAFE_LAYOUT = (memoryLayout instanceof UnsafeLayout);
@VisibleForTesting
public static boolean useUnsafeLayout() {
return USE_UNSAFE_LAYOUT;
}

View File

@ -28,7 +28,6 @@ import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FSDataOutputStreamBuilder;
@ -41,12 +40,10 @@ import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.yetus.audience.InterfaceAudience;
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.collect.Lists;
/**
@ -347,7 +344,7 @@ public final class CommonFSUtils {
}
/**
* Returns the URI in the strig format
* Returns the URI in the string format
* @param c configuration
* @param p path
* @return - the URI's to string format
@ -360,7 +357,6 @@ public final class CommonFSUtils {
return null;
}
@VisibleForTesting
public static void setWALRootDir(final Configuration c, final Path root) {
c.set(HBASE_WAL_DIR, root.toString());
}

View File

@ -21,12 +21,10 @@ import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import org.apache.yetus.audience.InterfaceAudience;
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;
/**
@ -213,12 +211,10 @@ public class IdLock {
}
}
@VisibleForTesting
void assertMapEmpty() {
assert map.isEmpty();
}
@VisibleForTesting
public void waitForWaiters(long id, int numWaiters) throws InterruptedException {
for (Entry entry;;) {
entry = map.get(id);

View File

@ -28,8 +28,6 @@ import java.nio.charset.Charset;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Utility class that handles ordered byte arrays. That is, unlike
* {@link Bytes}, these methods produce byte arrays which maintain the sort
@ -368,7 +366,7 @@ public class OrderedBytes {
* @param comp Compliment the encoded value when {@code comp} is true.
* @return number of bytes written.
*/
@VisibleForTesting
@InterfaceAudience.Private
static int putVaruint64(PositionedByteRange dst, long val, boolean comp) {
int w, y, len = 0;
final int offset = dst.getOffset(), start = dst.getPosition();
@ -457,7 +455,7 @@ public class OrderedBytes {
* @param comp if true, parse the compliment of the value.
* @return the number of bytes consumed by this value.
*/
@VisibleForTesting
@InterfaceAudience.Private
static int lengthVaruint64(PositionedByteRange src, boolean comp) {
int a0 = (comp ? DESCENDING : ASCENDING).apply(src.peek()) & 0xff;
if (a0 <= 240) return 1;
@ -478,7 +476,7 @@ public class OrderedBytes {
* @param cmp if true, parse the compliment of the value.
* @return the number of bytes skipped.
*/
@VisibleForTesting
@InterfaceAudience.Private
static int skipVaruint64(PositionedByteRange src, boolean cmp) {
final int len = lengthVaruint64(src, cmp);
src.setPosition(src.getPosition() + len);
@ -490,7 +488,7 @@ public class OrderedBytes {
* encoded value when {@code comp} is true.
* @return the decoded value.
*/
@VisibleForTesting
@InterfaceAudience.Private
static long getVaruint64(PositionedByteRange src, boolean comp) {
assert src.getRemaining() >= lengthVaruint64(src, comp);
final long ret;
@ -547,7 +545,7 @@ public class OrderedBytes {
* From Phoenix's {@code NumberUtil}.
* @return new {@link BigDecimal} instance
*/
@VisibleForTesting
@InterfaceAudience.Private
static BigDecimal normalize(BigDecimal val) {
return null == val ? null : val.stripTrailingZeros().round(DEFAULT_MATH_CONTEXT);
}
@ -1013,7 +1011,7 @@ public class OrderedBytes {
/**
* Calculate the expected BlobVar decoded length based on encoded length.
*/
@VisibleForTesting
@InterfaceAudience.Private
static int blobVarDecodedLength(int len) {
return
((len

View File

@ -27,8 +27,6 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.util.StringUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Utility methods for reading, and building the ZooKeeper configuration.
*
@ -274,7 +272,6 @@ public final class ZKConfig {
* @param clientPort the default client port
* @return the string for a list of "server:port" separated by ","
*/
@VisibleForTesting
public static String standardizeZKQuorumServerString(String quorumStringInput,
String clientPort) {
String[] serverHosts = quorumStringInput.split(",");
@ -287,7 +284,6 @@ public final class ZKConfig {
// in this case, the clientPort would be ignored)
// (3). s1:p1,s2,s3:p3 (mix of (1) and (2) - if port is not specified in a server, it would use
// the clientPort; otherwise, it would use the specified port)
@VisibleForTesting
public static class ZKClusterKey {
private String quorumString;
private int clientPort;

View File

@ -42,7 +42,7 @@ import org.junit.runners.Parameterized.Parameters;
import org.junit.runners.model.Statement;
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.collect.Iterables;
import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
@ -108,7 +108,6 @@ public final class HBaseClassTestRule implements TestRule {
* @return the number of parameters for this given test class. If the test is not parameterized or
* if there is any issue determining the number of parameters, returns 1.
*/
@VisibleForTesting
static int getNumParameters(Class<?> clazz) {
RunWith[] runWiths = clazz.getAnnotationsByType(RunWith.class);
boolean testParameterized = runWiths != null && Arrays.stream(runWiths).anyMatch(

View File

@ -16,6 +16,7 @@
* limitations under the License.
*/
package org.apache.hadoop.hbase.util;
import java.io.File;
import java.io.IOException;
import java.net.BindException;
@ -26,7 +27,7 @@ import org.apache.kerby.kerberos.kerb.KrbException;
import org.apache.kerby.kerberos.kerb.server.SimpleKdcServer;
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;
/**
@ -58,7 +59,6 @@ public final class SimpleKdcServerUtil {
* @return A running SimpleKdcServer on loopback/'localhost' on a random port
* @see #getRunningSimpleKdcServer(File, Supplier)
*/
@VisibleForTesting
static SimpleKdcServer getRunningSimpleKdcServer(File testDir,
Supplier<Integer> randomPortGenerator, final boolean portClash)
throws KrbException, IOException {

View File

@ -79,7 +79,6 @@ 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;
/**
* Export an HBase table. Writes content to sequence files up in HDFS. Use
@ -103,7 +102,7 @@ public class Export extends ExportProtos.ExportService implements RegionCoproces
System.exit(response == null ? -1 : 0);
}
@VisibleForTesting
@InterfaceAudience.Private
static Map<byte[], Response> run(final Configuration conf, final String[] args) throws Throwable {
String[] otherArgs = new GenericOptionsParser(conf, args).getRemainingArgs();
if (!ExportUtils.isValidArguements(args)) {

View File

@ -38,8 +38,6 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* This class acts as an adapter to export the MetricRegistry's in the global registry. Each
* MetricRegistry will be registered or unregistered from the metric2 system. The collection will
@ -102,7 +100,6 @@ public final class GlobalMetricRegistriesAdapter {
return new GlobalMetricRegistriesAdapter();
}
@VisibleForTesting
public void stop() {
stopped.set(true);
}

View File

@ -25,8 +25,6 @@ import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Implementation of {@link MetricsTableLatencies} to track latencies for one table in a
* RegionServer.
@ -36,7 +34,6 @@ public class MetricsTableLatenciesImpl extends BaseSourceImpl implements Metrics
private final HashMap<TableName,TableHistograms> histogramsByTable = new HashMap<>();
@VisibleForTesting
public static class TableHistograms {
final MetricHistogram getTimeHisto;
final MetricHistogram incrementTimeHisto;
@ -120,7 +117,6 @@ public class MetricsTableLatenciesImpl extends BaseSourceImpl implements Metrics
}
}
@VisibleForTesting
public static String qualifyMetricsName(TableName tableName, String metric) {
StringBuilder sb = new StringBuilder();
sb.append("Namespace_").append(tableName.getNamespaceAsString());
@ -129,7 +125,6 @@ public class MetricsTableLatenciesImpl extends BaseSourceImpl implements Metrics
return sb.toString();
}
@VisibleForTesting
public TableHistograms getOrCreateTableHistogram(String tableName) {
// TODO Java8's ConcurrentHashMap#computeIfAbsent would be stellar instead
final TableName tn = TableName.valueOf(tableName);

View File

@ -30,8 +30,6 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* JMX caches the beans that have been exported; even after the values are removed from hadoop's
* metrics system the keys and old values will still remain. This class stops and restarts the
@ -75,7 +73,6 @@ public final class JmxCacheBuster {
* Stops the clearing of JMX metrics and restarting the Hadoop metrics system. This is needed for
* some test environments where we manually inject sources or sinks dynamically.
*/
@VisibleForTesting
public static void stop() {
stopped.set(true);
ScheduledFuture future = fut.get();
@ -86,7 +83,6 @@ public final class JmxCacheBuster {
* Restarts the stopped service.
* @see #stop()
*/
@VisibleForTesting
public static void restart() {
stopped.set(false);
}

View File

@ -27,8 +27,6 @@ import java.util.Map;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Implementation of the Cormode, Korn, Muthukrishnan, and Srivastava algorithm
* for streaming calculation of targeted high-percentile epsilon-approximate
@ -257,7 +255,6 @@ public class MetricSampleQuantiles {
*
* @return count current number of samples
*/
@VisibleForTesting
synchronized public int getSampleCount() {
return samples.size();
}

View File

@ -67,7 +67,6 @@ 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.common.collect.Lists;
import org.apache.hbase.thirdparty.org.eclipse.jetty.http.HttpVersion;
@ -174,7 +173,6 @@ public class HttpServer implements FilterContainer {
private final List<ListenerInfo> listeners = Lists.newArrayList();
@VisibleForTesting
public List<ServerConnector> getServerConnectors() {
return listeners.stream().map(info -> info.listener).collect(Collectors.toList());
}
@ -1122,7 +1120,6 @@ public class HttpServer implements FilterContainer {
* Open the main listener for the server
* @throws Exception if the listener cannot be opened or the appropriate port is already in use
*/
@VisibleForTesting
void openListeners() throws Exception {
for (ListenerInfo li : listeners) {
ServerConnector listener = li.listener;

View File

@ -24,6 +24,7 @@ import java.io.InputStreamReader;
import java.io.PrintWriter;
import java.net.URL;
import java.net.URLConnection;
import java.nio.charset.StandardCharsets;
import java.util.Objects;
import java.util.regex.Pattern;
import javax.net.ssl.HttpsURLConnection;
@ -47,9 +48,6 @@ 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.Charsets;
/**
* Change log level in runtime.
*/
@ -89,7 +87,6 @@ public final class LogLevel {
protocol.equals(PROTOCOL_HTTPS)));
}
@VisibleForTesting
static class CLI extends Configured implements Tool {
private Operations operation = Operations.UNKNOWN;
private String protocol;
@ -289,7 +286,7 @@ public final class LogLevel {
// read from the servlet
try (InputStreamReader streamReader =
new InputStreamReader(connection.getInputStream(), Charsets.UTF_8);
new InputStreamReader(connection.getInputStream(), StandardCharsets.UTF_8);
BufferedReader bufferedReader = new BufferedReader(streamReader)) {
bufferedReader.lines().filter(Objects::nonNull).filter(line -> line.startsWith(MARKER))
.forEach(line -> System.out.println(TAG.matcher(line).replaceAll("")));

View File

@ -19,11 +19,9 @@
package org.apache.hadoop.hbase.mapred;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.util.ProgramDriver;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.util.ProgramDriver;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Driver for hbase mapreduce jobs. Select which to run by passing name of job
@ -35,7 +33,7 @@ public class Driver {
private static ProgramDriver pgd = new ProgramDriver();
@VisibleForTesting
@InterfaceAudience.Private
static void setProgramDriver(ProgramDriver pgd0) {
pgd = pgd0;
}

View File

@ -27,9 +27,6 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.CompareOperator;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.filter.IncompatibleFilterException;
@ -37,10 +34,12 @@ import org.apache.hadoop.hbase.filter.PrefixFilter;
import org.apache.hadoop.hbase.filter.RegexStringComparator;
import org.apache.hadoop.hbase.filter.RowFilter;
import org.apache.hadoop.hbase.security.visibility.Authorizations;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Triple;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Some helper methods are used by {@link org.apache.hadoop.hbase.mapreduce.Export}
@ -106,7 +105,6 @@ public final class ExportUtils {
return new Triple<>(TableName.valueOf(args[0]), getScanFromCommandLine(conf, args), new Path(args[1]));
}
@VisibleForTesting
static Scan getScanFromCommandLine(Configuration conf, String[] args) throws IOException {
Scan s = new Scan();
// Optional arguments.

View File

@ -90,8 +90,6 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Writes HFiles. Passed Cells must arrive in order.
* Writes current time as the sequence id for the file. Sets the major compacted
@ -681,7 +679,7 @@ public class HFileOutputFormat2
* @param conf to read the serialized values from
* @return a map from column family to the configured compression algorithm
*/
@VisibleForTesting
@InterfaceAudience.Private
static Map<byte[], Algorithm> createFamilyCompressionMap(Configuration
conf) {
Map<byte[], String> stringMap = createFamilyConfValueMap(conf,
@ -701,7 +699,7 @@ public class HFileOutputFormat2
* @param conf to read the serialized values from
* @return a map from column family to the the configured bloom filter type
*/
@VisibleForTesting
@InterfaceAudience.Private
static Map<byte[], BloomType> createFamilyBloomTypeMap(Configuration conf) {
Map<byte[], String> stringMap = createFamilyConfValueMap(conf,
BLOOM_TYPE_FAMILIES_CONF_KEY);
@ -720,7 +718,7 @@ public class HFileOutputFormat2
* @param conf to read the serialized values from
* @return a map from column family to the the configured bloom filter param
*/
@VisibleForTesting
@InterfaceAudience.Private
static Map<byte[], String> createFamilyBloomParamMap(Configuration conf) {
return createFamilyConfValueMap(conf, BLOOM_PARAM_FAMILIES_CONF_KEY);
}
@ -733,7 +731,7 @@ public class HFileOutputFormat2
* @param conf to read the serialized values from
* @return a map from column family to the configured block size
*/
@VisibleForTesting
@InterfaceAudience.Private
static Map<byte[], Integer> createFamilyBlockSizeMap(Configuration conf) {
Map<byte[], String> stringMap = createFamilyConfValueMap(conf,
BLOCK_SIZE_FAMILIES_CONF_KEY);
@ -753,7 +751,7 @@ public class HFileOutputFormat2
* @return a map from column family to HFileDataBlockEncoder for the
* configured data block type for the family
*/
@VisibleForTesting
@InterfaceAudience.Private
static Map<byte[], DataBlockEncoding> createFamilyDataBlockEncodingMap(
Configuration conf) {
Map<byte[], String> stringMap = createFamilyConfValueMap(conf,
@ -818,7 +816,7 @@ public class HFileOutputFormat2
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value =
"RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE")
@VisibleForTesting
@InterfaceAudience.Private
static String serializeColumnFamilyAttribute(Function<ColumnFamilyDescriptor, String> fn,
List<TableDescriptor> allTables)
throws UnsupportedEncodingException {
@ -849,7 +847,7 @@ public class HFileOutputFormat2
* Serialize column family to compression algorithm map to configuration.
* Invoked while configuring the MR job for incremental load.
*/
@VisibleForTesting
@InterfaceAudience.Private
static Function<ColumnFamilyDescriptor, String> compressionDetails = familyDescriptor ->
familyDescriptor.getCompressionType().getName();
@ -857,7 +855,7 @@ public class HFileOutputFormat2
* Serialize column family to block size map to configuration. Invoked while
* configuring the MR job for incremental load.
*/
@VisibleForTesting
@InterfaceAudience.Private
static Function<ColumnFamilyDescriptor, String> blockSizeDetails = familyDescriptor -> String
.valueOf(familyDescriptor.getBlocksize());
@ -865,7 +863,7 @@ public class HFileOutputFormat2
* Serialize column family to bloom type map to configuration. Invoked while
* configuring the MR job for incremental load.
*/
@VisibleForTesting
@InterfaceAudience.Private
static Function<ColumnFamilyDescriptor, String> bloomTypeDetails = familyDescriptor -> {
String bloomType = familyDescriptor.getBloomFilterType().toString();
if (bloomType == null) {
@ -878,7 +876,7 @@ public class HFileOutputFormat2
* Serialize column family to bloom param map to configuration. Invoked while
* configuring the MR job for incremental load.
*/
@VisibleForTesting
@InterfaceAudience.Private
static Function<ColumnFamilyDescriptor, String> bloomParamDetails = familyDescriptor -> {
BloomType bloomType = familyDescriptor.getBloomFilterType();
String bloomParam = "";
@ -892,7 +890,7 @@ public class HFileOutputFormat2
* Serialize column family to data block encoding map to configuration.
* Invoked while configuring the MR job for incremental load.
*/
@VisibleForTesting
@InterfaceAudience.Private
static Function<ColumnFamilyDescriptor, String> dataBlockEncodingDetails = familyDescriptor -> {
DataBlockEncoding encoding = familyDescriptor.getDataBlockEncoding();
if (encoding == null) {

View File

@ -29,7 +29,6 @@ import org.apache.hadoop.mapreduce.Job;
import java.io.IOException;
import java.nio.charset.Charset;
import java.util.List;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Create 3 level tree directory, first level is using table name as parent
@ -45,7 +44,6 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
* -columnFamilyName2
*/
@InterfaceAudience.Public
@VisibleForTesting
public class MultiTableHFileOutputFormat extends HFileOutputFormat2 {
private static final Logger LOG = LoggerFactory.getLogger(MultiTableHFileOutputFormat.class);

View File

@ -51,7 +51,6 @@ import org.apache.hadoop.util.StringUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* A base for {@link TableInputFormat}s. Receives a {@link Connection}, a {@link TableName},
@ -600,7 +599,7 @@ public abstract class TableInputFormatBase
this.connection = connection;
}
@VisibleForTesting
@InterfaceAudience.Private
protected RegionSizeCalculator createRegionSizeCalculator(RegionLocator locator, Admin admin)
throws IOException {
return new RegionSizeCalculator(locator, admin);

View File

@ -36,7 +36,6 @@ import org.apache.hadoop.util.StringUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Iterate over an HBase table data, return (ImmutableBytesWritable, Result)
@ -50,8 +49,9 @@ public class TableRecordReaderImpl {
private static final Logger LOG = LoggerFactory.getLogger(TableRecordReaderImpl.class);
// HBASE_COUNTER_GROUP_NAME is the name of mapreduce counter group for HBase
@VisibleForTesting
@InterfaceAudience.Private
static final String HBASE_COUNTER_GROUP_NAME = "HBaseCounters";
private ResultScanner scanner = null;
private Scan scan = null;
private Scan currentScan = null;

View File

@ -42,8 +42,6 @@ import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* TableSnapshotInputFormat allows a MapReduce job to run over a table snapshot. The job
* bypasses HBase servers, and directly accesses the underlying files (hfile, recovered edits,
@ -142,7 +140,7 @@ public class TableSnapshotInputFormat extends InputFormat<ImmutableBytesWritable
}
}
@VisibleForTesting
@InterfaceAudience.Private
static class TableSnapshotRegionRecordReader extends
RecordReader<ImmutableBytesWritable, Result> {
private TableSnapshotInputFormatImpl.RecordReader delegate =

View File

@ -69,8 +69,6 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* This map-only job compares the data from a local table with a remote one.
* Every cell is compared and must have exactly the same keys (even timestamp)
@ -517,7 +515,6 @@ public class VerifyReplication extends Configured implements Tool {
scan.setStopRow(stopRow);
}
@VisibleForTesting
public boolean doCommandLine(final String[] args) {
if (args.length < 2) {
printUsage(null);

View File

@ -28,8 +28,6 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
@InterfaceAudience.Private
public final class MetricRegistriesLoader {
private static final Logger LOG = LoggerFactory.getLogger(MetricRegistries.class);
@ -57,7 +55,6 @@ public final class MetricRegistriesLoader {
* implementation will be loaded.
* @return A {@link MetricRegistries} implementation.
*/
@VisibleForTesting
static MetricRegistries load(List<MetricRegistries> availableImplementations) {
if (availableImplementations.size() == 1) {

View File

@ -33,8 +33,6 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
/**
@ -591,7 +589,6 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
/**
* Called by the ProcedureExecutor to assign the ID to the newly created procedure.
*/
@VisibleForTesting
protected void setProcId(long procId) {
this.procId = procId;
this.submittedTime = EnvironmentEdgeManager.currentTime();
@ -612,12 +609,10 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
/**
* Called by the ProcedureExecutor to set the value to the newly created procedure.
*/
@VisibleForTesting
protected void setNonceKey(NonceKey nonceKey) {
this.nonceKey = nonceKey;
}
@VisibleForTesting
public void setOwner(String owner) {
this.owner = StringUtils.isEmpty(owner) ? null : owner;
}
@ -787,7 +782,6 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
return false;
}
@VisibleForTesting
protected synchronized void setState(final ProcedureState state) {
this.state = state;
updateTimestamp();

View File

@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.procedure2;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Basic ProcedureEvent that contains an "object", which can be a description or a reference to the
@ -126,7 +125,6 @@ public class ProcedureEvent<T> {
* Access to suspendedProcedures is 'synchronized' on this object, but it's fine to return it
* here for tests.
*/
@VisibleForTesting
public ProcedureDeque getSuspendedProcedures() {
return suspendedProcedures;
}

View File

@ -56,7 +56,6 @@ import org.apache.yetus.audience.InterfaceAudience;
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.common.util.concurrent.ThreadFactoryBuilder;
@ -639,7 +638,6 @@ public class ProcedureExecutor<TEnvironment> {
workerMonitorExecutor.sendStopSignal();
}
@VisibleForTesting
public void join() {
assert !isRunning() : "expected not running";
@ -1330,12 +1328,10 @@ public class ProcedureExecutor<TEnvironment> {
return procId;
}
@VisibleForTesting
protected long getLastProcId() {
return lastProcId.get();
}
@VisibleForTesting
public Set<Long> getActiveProcIds() {
return procedures.keySet();
}
@ -1930,17 +1926,14 @@ public class ProcedureExecutor<TEnvironment> {
return rollbackStack.get(rootProcId);
}
@VisibleForTesting
ProcedureScheduler getProcedureScheduler() {
return scheduler;
}
@VisibleForTesting
int getCompletedSize() {
return completed.size();
}
@VisibleForTesting
public IdLock getProcExecutionLock() {
return procExecutionLock;
}

View File

@ -22,8 +22,6 @@ import java.util.List;
import java.util.concurrent.TimeUnit;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Keep track of the runnable procedures
*/
@ -125,7 +123,6 @@ public interface ProcedureScheduler {
* Returns the number of elements in this queue.
* @return the number of elements in this queue.
*/
@VisibleForTesting
int size();
/**
@ -133,6 +130,5 @@ public interface ProcedureScheduler {
* Used for testing failure and recovery. To emulate server crash/restart,
* {@link ProcedureExecutor} resets its own state and calls clear() on scheduler.
*/
@VisibleForTesting
void clear();
}

View File

@ -23,8 +23,6 @@ import java.util.List;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Simple scheduler for procedures
*/
@ -47,7 +45,6 @@ public class SimpleProcedureScheduler extends AbstractProcedureScheduler {
return runnables.poll();
}
@VisibleForTesting
@Override
public void clear() {
schedLock();

View File

@ -28,8 +28,6 @@ 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.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.StateMachineProcedureData;
/**
@ -73,7 +71,6 @@ public abstract class StateMachineProcedure<TEnvironment, TState>
*/
private int previousState;
@VisibleForTesting
public enum Flow {
HAS_MORE_STATE,
NO_MORE_STATE,
@ -282,7 +279,6 @@ public abstract class StateMachineProcedure<TEnvironment, TState>
* sequentially. Some procedures may skip steps/ states, some may add intermediate steps in
* future.
*/
@VisibleForTesting
public int getCurrentStateId() {
return getStateId(getCurrentState());
}

View File

@ -59,7 +59,6 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.org.apache.commons.collections4.queue.CircularFifoQueue;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALHeader;
@ -240,7 +239,6 @@ public class WALProcedureStore extends ProcedureStoreBase {
leaseRecovery);
}
@VisibleForTesting
public WALProcedureStore(final Configuration conf, final Path walDir, final Path walArchiveDir,
final LeaseRecovery leaseRecovery) throws IOException {
this.conf = conf;
@ -984,7 +982,6 @@ public class WALProcedureStore extends ProcedureStoreBase {
return (System.currentTimeMillis() - lastRollTs.get());
}
@VisibleForTesting
void periodicRollForTesting() throws IOException {
lock.lock();
try {
@ -994,7 +991,6 @@ public class WALProcedureStore extends ProcedureStoreBase {
}
}
@VisibleForTesting
public boolean rollWriterForTesting() throws IOException {
lock.lock();
try {
@ -1004,7 +1000,6 @@ public class WALProcedureStore extends ProcedureStoreBase {
}
}
@VisibleForTesting
void removeInactiveLogsForTesting() throws Exception {
lock.lock();
try {
@ -1058,7 +1053,6 @@ public class WALProcedureStore extends ProcedureStoreBase {
return true;
}
@VisibleForTesting
boolean rollWriter(long logId) throws IOException {
assert logId > flushLogId : "logId=" + logId + " flushLogId=" + flushLogId;
assert lock.isHeldByCurrentThread() : "expected to be the lock owner. " + lock.isLocked();
@ -1257,7 +1251,6 @@ public class WALProcedureStore extends ProcedureStoreBase {
return this.walDir;
}
@VisibleForTesting
Path getWalArchiveDir() {
return this.walArchiveDir;
}

View File

@ -27,8 +27,6 @@ import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* This provides an class for maintaining a set of peer clusters. These peers are remote slave
* clusters that data is replicated to.
@ -59,7 +57,6 @@ public class ReplicationPeers {
}
}
@VisibleForTesting
public ReplicationPeerStorage getPeerStorage() {
return this.peerStorage;
}

View File

@ -23,15 +23,13 @@ import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
/**
* ZK based replication peer storage.
@ -69,12 +67,10 @@ public class ZKReplicationPeerStorage extends ZKReplicationStorageBase
this.peersZNode = ZNodePaths.joinZNode(replicationZNode, peersZNodeName);
}
@VisibleForTesting
public String getPeerStateNode(String peerId) {
return ZNodePaths.joinZNode(getPeerNode(peerId), peerStateNodeName);
}
@VisibleForTesting
public String getPeerNode(String peerId) {
return ZNodePaths.joinZNode(peersZNode, peerId);
}

View File

@ -51,7 +51,6 @@ import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
/**
@ -103,7 +102,6 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
*/
private final String hfileRefsZNode;
@VisibleForTesting
final String regionsZNode;
public ZKReplicationQueueStorage(ZKWatcher zookeeper, Configuration conf) {
@ -158,7 +156,6 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
* @return ZNode path to persist the max sequence id that we've pushed for the given region and
* peer.
*/
@VisibleForTesting
String getSerialReplicationRegionPeerNode(String encodedRegionName, String peerId) {
if (encodedRegionName == null || encodedRegionName.length() != RegionInfo.MD5_HEX_LENGTH) {
throw new IllegalArgumentException(
@ -264,7 +261,6 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
* Return the {lastPushedSequenceId, ZNodeDataVersion} pair. if ZNodeDataVersion is -1, it means
* that the ZNode does not exist.
*/
@VisibleForTesting
protected Pair<Long, Integer> getLastSequenceIdWithVersion(String encodedRegionName,
String peerId) throws KeeperException {
Stat stat = new Stat();
@ -503,7 +499,6 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
}
// will be overridden in UTs
@VisibleForTesting
protected int getQueuesZNodeCversion() throws KeeperException {
Stat stat = new Stat();
ZKUtil.getDataNoWatch(this.zookeeper, this.queuesZNode, stat);
@ -641,7 +636,6 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
}
// will be overridden in UTs
@VisibleForTesting
protected int getHFileRefsZNodeCversion() throws ReplicationException {
Stat stat = new Stat();
try {

View File

@ -88,14 +88,12 @@ import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.security.access.AccessChecker;
import org.apache.hadoop.hbase.security.access.Permission.Action;
import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
import org.apache.yetus.audience.InterfaceAudience;
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.collect.Maps;
import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
// TODO: Encapsulate MasterObserver functions into separate subclass.
@CoreCoprocessor
@InterfaceAudience.Private
@ -151,7 +149,6 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver {
return groupInfoManager;
}
@VisibleForTesting
RSGroupAdminServer getGroupAdminServer() {
return groupAdminServer;
}

View File

@ -45,13 +45,11 @@ import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.master.assignment.RegionStateNode;
import org.apache.hadoop.hbase.net.Address;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
import org.apache.yetus.audience.InterfaceAudience;
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.collect.Maps;
/**
* Service to support Region Server Grouping (HBase-6721).
*/
@ -643,7 +641,6 @@ public class RSGroupAdminServer implements RSGroupAdmin {
* parent region cases. This method is invoked by {@link #balanceRSGroup}
* @return A clone of current assignments for this group.
*/
@VisibleForTesting
Map<TableName, Map<ServerName, List<RegionInfo>>> getRSGroupAssignmentsByTable(
TableStateManager tableStateManager, String groupName) throws IOException {
Map<TableName, Map<ServerName, List<RegionInfo>>> result = Maps.newHashMap();

View File

@ -28,7 +28,6 @@ import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.TreeMap;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterMetrics;
import org.apache.hadoop.hbase.HBaseIOException;
@ -48,7 +47,6 @@ import org.apache.yetus.audience.InterfaceAudience;
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.collect.ArrayListMultimap;
import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap;
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
@ -414,7 +412,6 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
return false;
}
@VisibleForTesting
public void setRsGroupInfoManager(RSGroupInfoManager rsGroupInfoManager) {
this.rsGroupInfoManager = rsGroupInfoManager;
}

View File

@ -75,16 +75,14 @@ import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
import org.apache.hadoop.util.Shell;
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
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.collect.Lists;
import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
/**
* This is an implementation of {@link RSGroupInfoManager} which makes use of an HBase table as the
* persistence store for the group information. It also makes use of zookeeper to store group
@ -140,7 +138,6 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager {
new ServerEventsListenerThread();
/** Get rsgroup table mapping script */
@VisibleForTesting
RSGroupMappingScript script;
// Package visibility for testing

View File

@ -31,7 +31,7 @@ import org.apache.hadoop.util.ToolRunner;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLineParser;
import org.apache.hbase.thirdparty.org.apache.commons.cli.DefaultParser;
@ -48,7 +48,7 @@ public class RSGroupMajorCompactionTTL extends MajorCompactorTTL {
private static final Logger LOG = LoggerFactory.getLogger(RSGroupMajorCompactionTTL.class);
@VisibleForTesting
@InterfaceAudience.Private
RSGroupMajorCompactionTTL() {
super();
}

View File

@ -25,17 +25,15 @@ import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockService;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Lock for HBase Entity either a Table, a Namespace, or Regions.
@ -145,12 +143,12 @@ public class EntityLock {
return sb.toString();
}
@VisibleForTesting
@InterfaceAudience.Private
void setTestingSleepTime(long timeInMillis) {
testingSleepTime = timeInMillis;
}
@VisibleForTesting
@InterfaceAudience.Private
LockHeartbeatWorker getWorker() {
return worker;
}

View File

@ -20,7 +20,6 @@
package org.apache.hadoop.hbase.client.locking;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
@ -30,7 +29,6 @@ import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockService;
@ -89,7 +87,7 @@ public class LockServiceClient {
return new EntityLock(conf, stub, lockRequest, abort);
}
@VisibleForTesting
@InterfaceAudience.Private
public static LockRequest buildLockRequest(final LockType type,
final String namespace, final TableName tableName, final List<RegionInfo> regionInfos,
final String description, final long nonceGroup, final long nonce) {

View File

@ -29,8 +29,6 @@ import org.apache.hadoop.hbase.master.SplitLogManager.ResubmitDirective;
import org.apache.hadoop.hbase.master.SplitLogManager.Task;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Coordination for SplitLogManager. It creates and works with tasks for split log operations<BR>
* Manager prepares task by calling {@link #prepareTask} and submit it by
@ -147,6 +145,5 @@ public interface SplitLogManagerCoordination {
* Support method to init constants such as timeout. Mostly required for UTs.
* @throws IOException
*/
@VisibleForTesting
void init() throws IOException;
}

View File

@ -25,7 +25,6 @@ import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.regionserver.SplitLogWorker;
import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Coordinated operations for {@link SplitLogWorker} and
@ -94,7 +93,6 @@ public interface SplitLogWorkerCoordination {
* Used by unit tests to check how many tasks were processed
* @return number of tasks
*/
@VisibleForTesting
int getTaskReadySeq();
/**

View File

@ -58,7 +58,6 @@ import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* ZooKeeper based implementation of
@ -742,7 +741,6 @@ public class ZKSplitLogManagerCoordination extends ZKListener implements
/**
* Temporary function that is used by unit tests only
*/
@VisibleForTesting
public void setIgnoreDeleteForTesting(boolean b) {
ignoreZKDeleteForTesting = b;
}

View File

@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.security.User;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.util.CoprocessorClassLoader;
import org.apache.hadoop.hbase.util.SortedList;
@ -253,7 +252,6 @@ public abstract class CoprocessorHost<C extends Coprocessor, E extends Coprocess
}
}
@VisibleForTesting
public void load(Class<? extends C> implClass, int priority, Configuration conf)
throws IOException {
E env = checkAndLoadInstance(implClass, priority, conf);
@ -325,7 +323,6 @@ public abstract class CoprocessorHost<C extends Coprocessor, E extends Coprocess
return null;
}
@VisibleForTesting
public <T extends C> T findCoprocessor(Class<T> cls) {
for (E env: coprocEnvironments) {
if (cls.isAssignableFrom(env.getInstance().getClass())) {
@ -360,7 +357,6 @@ public abstract class CoprocessorHost<C extends Coprocessor, E extends Coprocess
* @param className the class name
* @return the coprocessor, or null if not found
*/
@VisibleForTesting
public E findCoprocessorEnvironment(String className) {
for (E env: coprocEnvironments) {
if (env.getInstance().getClass().getName().equals(className) ||

View File

@ -25,8 +25,6 @@ import org.apache.hadoop.hbase.metrics.MetricRegistries;
import org.apache.hadoop.hbase.metrics.MetricRegistry;
import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Utility class for tracking metrics for various types of coprocessors. Each coprocessor instance
* creates its own MetricRegistry which is exported as an individual MetricSource. MetricRegistries
@ -75,7 +73,6 @@ public class MetricsCoprocessor {
.toString();
}
@VisibleForTesting
static MetricRegistryInfo createRegistryInfoForMasterCoprocessor(String clazz) {
return new MetricRegistryInfo(
suffix(MASTER_COPROC_METRICS_NAME, clazz),
@ -88,7 +85,6 @@ public class MetricsCoprocessor {
return MetricRegistries.global().create(createRegistryInfoForMasterCoprocessor(clazz));
}
@VisibleForTesting
static MetricRegistryInfo createRegistryInfoForRSCoprocessor(String clazz) {
return new MetricRegistryInfo(
suffix(RS_COPROC_METRICS_NAME, clazz),
@ -101,7 +97,6 @@ public class MetricsCoprocessor {
return MetricRegistries.global().create(createRegistryInfoForRSCoprocessor(clazz));
}
@VisibleForTesting
public static MetricRegistryInfo createRegistryInfoForRegionCoprocessor(String clazz) {
return new MetricRegistryInfo(
suffix(REGION_COPROC_METRICS_NAME, clazz),
@ -114,7 +109,6 @@ public class MetricsCoprocessor {
return MetricRegistries.global().create(createRegistryInfoForRegionCoprocessor(clazz));
}
@VisibleForTesting
public static MetricRegistryInfo createRegistryInfoForWALCoprocessor(String clazz) {
return new MetricRegistryInfo(
suffix(WAL_COPROC_METRICS_NAME, clazz),

View File

@ -23,8 +23,6 @@ import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.security.User;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* This is the only implementation of {@link ObserverContext}, which serves as the interface for
* third-party Coprocessor developers.
@ -98,7 +96,6 @@ public class ObserverContextImpl<E extends CoprocessorEnvironment> implements Ob
* @return An instance of <code>ObserverContext</code> with the environment set
*/
@Deprecated
@VisibleForTesting
// TODO: Remove this method, ObserverContext should not depend on RpcServer
public static <E extends CoprocessorEnvironment> ObserverContext<E> createAndPrepare(E env) {
ObserverContextImpl<E> ctx = new ObserverContextImpl<>(RpcServer.getRequestUser().orElse(null));

View File

@ -32,13 +32,11 @@ import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.hbase.monitoring.ThreadMonitoring;
import org.apache.yetus.audience.InterfaceAudience;
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.collect.Lists;
import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ListenableFuture;
@ -85,7 +83,6 @@ public class ExecutorService {
* started with the same name, this throws a RuntimeException.
* @param name Name of the service to start.
*/
@VisibleForTesting
public void startExecutorService(String name, int maxThreads) {
if (this.executorMap.get(name) != null) {
throw new RuntimeException("An executor service with the name " + name +
@ -126,7 +123,6 @@ public class ExecutorService {
return executor;
}
@VisibleForTesting
public ThreadPoolExecutor getExecutorThreadPool(final ExecutorType type) {
return getExecutor(type).getThreadPoolExecutor();
}

View File

@ -22,6 +22,7 @@ import static org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper.FAVORE
import static org.apache.hadoop.hbase.favored.FavoredNodesPlan.Position.PRIMARY;
import static org.apache.hadoop.hbase.favored.FavoredNodesPlan.Position.SECONDARY;
import static org.apache.hadoop.hbase.favored.FavoredNodesPlan.Position.TERTIARY;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
@ -42,7 +43,7 @@ import org.apache.hadoop.net.NetUtils;
import org.apache.yetus.audience.InterfaceAudience;
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.collect.Lists;
import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
@ -95,7 +96,6 @@ public class FavoredNodesManager {
datanodeDataTransferPort= getDataNodePort();
}
@VisibleForTesting
public int getDataNodePort() {
HdfsConfiguration.init();
@ -282,7 +282,6 @@ public class FavoredNodesManager {
}
}
@VisibleForTesting
public synchronized Set<RegionInfo> getRegionsOfFavoredNode(ServerName serverName) {
Set<RegionInfo> regionInfos = Sets.newHashSet();

Some files were not shown because too many files have changed in this diff Show More