diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
index 4c341c5dba6..669b2271fc6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
@@ -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;
-
/**
*
* Read/write operations on hbase:meta
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 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 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 parents) {
ByteArrayOutputStream bos = new ByteArrayOutputStream();
Iterator iter = parents.iterator();
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorImpl.java
index 7aa9597c594..a7e5f3ff6d1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorImpl.java
@@ -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();
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
index ad8ef0a520a..b48729f7461 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
@@ -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;
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
index 2627336a4ef..12e735b8ae3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
@@ -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) {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
index de7449bf285..8cd046f2006 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
@@ -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
AsyncRequestFutureImpl createAsyncRequestFuture(
AsyncProcessTask task, List 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 createCaller(
CancellableRegionServerCallable callable, int rpcTimeout) {
return rpcCallerFactory. newCaller(checkRpcTimeout(rpcTimeout));
}
-
/**
* Creates the server error tracker to use inside process.
* Currently, to preserve the main assumption about current retries, and to work well with
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
index 9e1d5e8f5a6..d50070ae8a9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
@@ -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;
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java
index e46a50e7619..d2486cc1cf0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java
@@ -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 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 callsInProgress;
- @VisibleForTesting
SingleServerRequestRunnable(
MultiAction multiAction, int numAttempt, ServerName server,
Set callsInProgress) {
@@ -394,12 +390,10 @@ class AsyncRequestFutureImpl implements AsyncRequestFuture {
}
}
- @VisibleForTesting
protected Set getCallsInProgress() {
return callsInProgress;
}
- @VisibleForTesting
SingleServerRequestRunnable createSingleServerRequest(MultiAction multiAction, int numAttempt, ServerName server,
Set callsInProgress) {
return new SingleServerRequestRunnable(multiAction, numAttempt, server, callsInProgress);
@@ -920,13 +914,11 @@ class AsyncRequestFutureImpl 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 implements AsyncRequestFuture {
results[index] = result;
}
- @VisibleForTesting
long getNumberOfActionsInProgress() {
return actionsInProgress.get();
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableResultScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableResultScanner.java
index 21ff4b7acce..7d520099744 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableResultScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableResultScanner.java
@@ -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;
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
index f0c8da413d8..11ce17419f1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
@@ -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;
-
/**
*
* 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, Closeable {
private int remainder = undealtMutationCount.getAndSet(0);
private Mutation last = null;
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java
index ba5ee063a7f..0c832acdb37 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java
@@ -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 prefetchListener) {
this.prefetchListener = prefetchListener;
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
index e92d3f0ec01..c799e5b3872 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
@@ -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;
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java
index e809c4c5eb2..19a398b8c66 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java
@@ -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;
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 6fbc6caca2d..e9039c248fa 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -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);
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
index 8f9dc277f4d..d6f21944a1a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
@@ -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());
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index b643d9422bb..9994baf9de0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -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
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
index 7b81f442931..5f3afa1f263 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
@@ -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;
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
index 56c551f65d7..539b02dba3a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
@@ -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 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 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;
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
index 06582684c79..1a465798ed2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
@@ -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 {
*
* 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 getParsedMasterServers() {
return masterAddr2Stub.keySet();
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java
index e9f4c61f5a2..993c6caae1a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java
@@ -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> 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 rpcTimers =
+ protected final ConcurrentMap rpcTimers =
new ConcurrentHashMap<>(CAPACITY, LOAD_FACTOR, CONCURRENCY_LEVEL);
- @VisibleForTesting protected final ConcurrentMap rpcHistograms =
+ protected final ConcurrentMap rpcHistograms =
new ConcurrentHashMap<>(CAPACITY * 2 /* tracking both request and response sizes */,
LOAD_FACTOR, CONCURRENCY_LEVEL);
private final ConcurrentMap cacheDroppingExceptions =
new ConcurrentHashMap<>(CAPACITY, LOAD_FACTOR, CONCURRENCY_LEVEL);
- @VisibleForTesting protected final ConcurrentMap rpcCounters =
+ protected final ConcurrentMap rpcCounters =
new ConcurrentHashMap<>(CAPACITY, LOAD_FACTOR, CONCURRENCY_LEVEL);
MetricsConnection(String scope, Supplier 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;
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
index 0812f2ab25f..4b4f08410a2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
@@ -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 multi
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
setStub(getConnection().getClient(this.location.getServerName()));
}
- @VisibleForTesting
ServerName getServerName() {
return location.getServerName();
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PreemptiveFastFailInterceptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PreemptiveFastFailInterceptor.java
index a8406a452a4..e82de2ad754 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PreemptiveFastFailInterceptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PreemptiveFastFailInterceptor.java
@@ -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;
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index 9c9918e054d..a45003f6025 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -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 getRegionLocation(byte[] regionNameOrEncodedRegionName) {
if (regionNameOrEncodedRegionName == null) {
return failedFuture(new IllegalArgumentException("Passed region name can't be null"));
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
index 902e15eaa74..636ca374d3b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
@@ -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 {
callable.getScan().withStartRow(this.lastResult.getRow(), this.lastResult.mayHaveMoreCellsInRow());
}
- @VisibleForTesting
boolean isAnyRPCcancelled() {
return someRPCcancelled;
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SimpleRequestController.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SimpleRequestController.java
index e5e2a6dd3a3..54abc452a30 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SimpleRequestController.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SimpleRequestController.java
@@ -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 taskCounterPerRegion
= new ConcurrentSkipListMap<>(Bytes.BYTES_COMPARATOR);
- @VisibleForTesting
final ConcurrentMap 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 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);
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java
index 867d6f03825..35107cab2ba 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java
@@ -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;
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
index 722dc2aadb6..16d2f51f7bf 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
@@ -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;
-
/**
*
* 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 listReplicationPeers() throws IOException {
return admin.listReplicationPeers();
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ClientExceptionsUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ClientExceptionsUtil.java
index 6b1e251953b..2482a632ca8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ClientExceptionsUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ClientExceptionsUtil.java
@@ -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> getConnectionExceptionTypes() {
return CONNECTION_EXCEPTION_TYPES;
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
index abfa7e1d6e8..259a6cb09a4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
@@ -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 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
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
index bf5130475d2..40f225fd0c3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
@@ -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 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 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 implements RpcC
/**
* Blocking rpc channel that goes via hbase rpc.
*/
- @VisibleForTesting
public static class BlockingRpcChannelImplementation extends AbstractRpcChannel
implements BlockingRpcChannel {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java
index 22eca535e95..dd8f96bb2b9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java
@@ -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
* 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);
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
index 811bb2c21fd..1b1411c7b95 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
@@ -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) {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientFactory.java
index b6f9e385930..434795248c6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientFactory.java
@@ -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);
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java
index 50594c81e5c..2d03473b9f6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java
@@ -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);
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/ThrottleSettings.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/ThrottleSettings.java
index 3c25d6e4f94..d77d8d168c8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/ThrottleSettings.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/ThrottleSettings.java
@@ -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 this
*/
- @VisibleForTesting
QuotaProtos.ThrottleRequest getProto() {
return proto.toBuilder().build();
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index a56c90ca933..3e9046579f2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -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;
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java
index b4f3ccba9cd..a1475de8a1e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java
@@ -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;
}
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java
index 8c6a2d58895..15c6f7aacce 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java
@@ -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;
}
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
index 1dbbe43ad1d..bbbf776c9c5 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
@@ -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) {
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java
index 9dbb307df40..39c3ccc6919 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java
@@ -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);
}
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
index b7ddbab6c90..856480f1506 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
@@ -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.
*
@@ -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;
}
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java
index c26841dad1a..01bee5672ff 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java
@@ -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
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ScheduledChore.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ScheduledChore.java
index a009cf4ef12..1fb5b7e9e34 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ScheduledChore.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ScheduledChore.java
@@ -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();
}
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/conf/ConfigurationManager.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/conf/ConfigurationManager.java
index e8357ebfdad..ca9b1e7acc6 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/conf/ConfigurationManager.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/conf/ConfigurationManager.java
@@ -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
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBuffAllocator.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBuffAllocator.java
index b5dff1ba4cb..03e1f9b3034 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBuffAllocator.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBuffAllocator.java
@@ -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();
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/aes/AES.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/aes/AES.java
index 73b87135083..69f12f9ae06 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/aes/AES.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/aes/AES.java
@@ -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;
}
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/aes/CommonsCryptoAES.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/aes/CommonsCryptoAES.java
index a119c57f59d..19c2bd8ae06 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/aes/CommonsCryptoAES.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/aes/CommonsCryptoAES.java
@@ -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;
}
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java
index 3c16cffaef2..2f8a19b2eff 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java
@@ -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;
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hadoopbackport/ThrottledInputStream.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hadoopbackport/ThrottledInputStream.java
index a488185591c..68627c3b1c8 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hadoopbackport/ThrottledInputStream.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hadoopbackport/ThrottledInputStream.java
@@ -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) {
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/security/UserProvider.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/UserProvider.java
index efa18fb9f58..0c054ceaaa2 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/security/UserProvider.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/UserProvider.java
@@ -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;
}
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java
index 42d1bf4fa00..0c4c52f99fe 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java
@@ -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)) {
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
index ae87e32d781..3ff3390172d 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
@@ -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();
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
index 06a1df7bd33..3fe50927e5e 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
@@ -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 {
// 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 {
}
- @VisibleForTesting
+ @InterfaceAudience.Private
static Comparer lexicographicalComparerJavaImpl() {
return LexicographicalComparerHolder.PureJavaComparer.INSTANCE;
}
@@ -1547,7 +1543,7 @@ public class Bytes implements Comparable {
* 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 {
}
}
- @VisibleForTesting
+ @InterfaceAudience.Private
enum UnsafeComparer implements Comparer {
INSTANCE;
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ClassSize.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ClassSize.java
index 6f88c005cb8..ebb9001ce86 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ClassSize.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ClassSize.java
@@ -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;
}
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CommonFSUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CommonFSUtils.java
index 63c9f325e4a..06120524832 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CommonFSUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CommonFSUtils.java
@@ -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());
}
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/IdLock.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/IdLock.java
index 9e5692feebb..112af1ef85a 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/IdLock.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/IdLock.java
@@ -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);
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
index ff2fd453835..db85f9fb0f2 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
@@ -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
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java
index f324ec68cf5..008d32fbbe2 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java
@@ -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;
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java
index 16bce3e48b0..c30e32473b9 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java
@@ -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(
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/SimpleKdcServerUtil.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/SimpleKdcServerUtil.java
index 32ffa7fcae2..79aae9b7879 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/SimpleKdcServerUtil.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/SimpleKdcServerUtil.java
@@ -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 randomPortGenerator, final boolean portClash)
throws KrbException, IOException {
diff --git a/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/coprocessor/Export.java b/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/coprocessor/Export.java
index b21d5c3f29a..38f5f434eb5 100644
--- a/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/coprocessor/Export.java
+++ b/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/coprocessor/Export.java
@@ -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 run(final Configuration conf, final String[] args) throws Throwable {
String[] otherArgs = new GenericOptionsParser(conf, args).getRemainingArgs();
if (!ExportUtils.isValidArguements(args)) {
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/impl/GlobalMetricRegistriesAdapter.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/impl/GlobalMetricRegistriesAdapter.java
index 96fc95416a8..a816d497044 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/impl/GlobalMetricRegistriesAdapter.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/impl/GlobalMetricRegistriesAdapter.java
@@ -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);
}
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableLatenciesImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableLatenciesImpl.java
index 5e13a614ff0..dd143d4c6f5 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableLatenciesImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableLatenciesImpl.java
@@ -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 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);
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java
index fbcd9fc7eaf..88b491ba3ea 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java
@@ -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);
}
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/util/MetricSampleQuantiles.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/util/MetricSampleQuantiles.java
index 8d075589d4a..c1880f8203b 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/util/MetricSampleQuantiles.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/util/MetricSampleQuantiles.java
@@ -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();
}
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java
index 1863557529d..41844e75543 100644
--- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java
+++ b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java
@@ -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 listeners = Lists.newArrayList();
- @VisibleForTesting
public List 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;
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/log/LogLevel.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/log/LogLevel.java
index dede1f9d8fd..1fcfa1390c2 100644
--- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/log/LogLevel.java
+++ b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/log/LogLevel.java
@@ -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("")));
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/Driver.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/Driver.java
index da907cbff77..b1f71f057f2 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/Driver.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/Driver.java
@@ -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;
}
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/ExportUtils.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/ExportUtils.java
index dfa3d312d42..75feab5e4bb 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/ExportUtils.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/ExportUtils.java
@@ -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.
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
index a1012b3cdc1..b94d1818d0a 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
@@ -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 createFamilyCompressionMap(Configuration
conf) {
Map 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 createFamilyBloomTypeMap(Configuration conf) {
Map 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 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 createFamilyBlockSizeMap(Configuration conf) {
Map 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 createFamilyDataBlockEncodingMap(
Configuration conf) {
Map 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 fn,
List 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 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 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 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 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 dataBlockEncodingDetails = familyDescriptor -> {
DataBlockEncoding encoding = familyDescriptor.getDataBlockEncoding();
if (encoding == null) {
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableHFileOutputFormat.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableHFileOutputFormat.java
index cdc2f94c740..6410bf8726c 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableHFileOutputFormat.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableHFileOutputFormat.java
@@ -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);
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
index 345a2f6c5be..61672d49fb2 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
@@ -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);
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReaderImpl.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReaderImpl.java
index 0e3281fa64f..47a9ce471db 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReaderImpl.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReaderImpl.java
@@ -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;
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormat.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormat.java
index 4016808f107..24cbfcc3165 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormat.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormat.java
@@ -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 {
private TableSnapshotInputFormatImpl.RecordReader delegate =
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
index 7f8a71ef40a..451da8724b0 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
@@ -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);
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistriesLoader.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistriesLoader.java
index e17a28a5828..edc813d95b9 100644
--- a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistriesLoader.java
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistriesLoader.java
@@ -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 availableImplementations) {
if (availableImplementations.size() == 1) {
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
index cf42c880fd8..c0bbfce583a 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
@@ -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 implements Comparable implements Comparable implements Comparable {
* Access to suspendedProcedures is 'synchronized' on this object, but it's fine to return it
* here for tests.
*/
- @VisibleForTesting
public ProcedureDeque getSuspendedProcedures() {
return suspendedProcedures;
}
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
index 38d5e2b53d0..0932cf4eeab 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
@@ -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 {
workerMonitorExecutor.sendStopSignal();
}
- @VisibleForTesting
public void join() {
assert !isRunning() : "expected not running";
@@ -1330,12 +1328,10 @@ public class ProcedureExecutor {
return procId;
}
- @VisibleForTesting
protected long getLastProcId() {
return lastProcId.get();
}
- @VisibleForTesting
public Set getActiveProcIds() {
return procedures.keySet();
}
@@ -1930,17 +1926,14 @@ public class ProcedureExecutor {
return rollbackStack.get(rootProcId);
}
- @VisibleForTesting
ProcedureScheduler getProcedureScheduler() {
return scheduler;
}
- @VisibleForTesting
int getCompletedSize() {
return completed.size();
}
- @VisibleForTesting
public IdLock getProcExecutionLock() {
return procExecutionLock;
}
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureScheduler.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureScheduler.java
index d787cc0979c..72b2b284ca1 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureScheduler.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureScheduler.java
@@ -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();
}
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SimpleProcedureScheduler.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SimpleProcedureScheduler.java
index feab8be16c0..2b043d472d0 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SimpleProcedureScheduler.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SimpleProcedureScheduler.java
@@ -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();
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
index 46c4c5e545e..797381db4e2 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
@@ -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
*/
private int previousState;
- @VisibleForTesting
public enum Flow {
HAS_MORE_STATE,
NO_MORE_STATE,
@@ -282,7 +279,6 @@ public abstract class StateMachineProcedure
* sequentially. Some procedures may skip steps/ states, some may add intermediate steps in
* future.
*/
- @VisibleForTesting
public int getCurrentStateId() {
return getStateId(getCurrentState());
}
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
index e7d72bd6f1d..e59d5662afd 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
@@ -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;
}
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
index e754e387d43..ea5a7ac4c4a 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
@@ -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;
}
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java
index bbe65498738..4b3b70220c3 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java
@@ -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);
}
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
index 6f1f5a3d6a4..5c480bacdd8 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
@@ -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 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 {
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
index 0311451da56..55cf88d0e23 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
@@ -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;
}
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java
index 01cea18c8f2..73ed49bfd19 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java
@@ -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>> getRSGroupAssignmentsByTable(
TableStateManager tableStateManager, String groupName) throws IOException {
Map>> result = Maps.newHashMap();
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java
index 52f1681ca36..901aafa7a5a 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java
@@ -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;
}
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
index a09d2f91059..53c74914761 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
@@ -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
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupMajorCompactionTTL.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupMajorCompactionTTL.java
index d1b375181cc..6936c721137 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupMajorCompactionTTL.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupMajorCompactionTTL.java
@@ -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();
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/locking/EntityLock.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/locking/EntityLock.java
index e49fe8c8c93..b68a962d15a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/locking/EntityLock.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/locking/EntityLock.java
@@ -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;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/locking/LockServiceClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/locking/LockServiceClient.java
index 4229a7aa6fc..24f2835af8b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/locking/LockServiceClient.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/locking/LockServiceClient.java
@@ -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 regionInfos,
final String description, final long nonceGroup, final long nonce) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitLogManagerCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitLogManagerCoordination.java
index 33d8f2ca779..9508321a625 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitLogManagerCoordination.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitLogManagerCoordination.java
@@ -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
* 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;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitLogWorkerCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitLogWorkerCoordination.java
index a9fae4640d5..5452578a2c2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitLogWorkerCoordination.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitLogWorkerCoordination.java
@@ -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();
/**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java
index 3ff722a5c85..b6256ad7005 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java
@@ -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;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
index 7e0f2ae2b86..44250763005 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
@@ -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 implClass, int priority, Configuration conf)
throws IOException {
E env = checkAndLoadInstance(implClass, priority, conf);
@@ -325,7 +323,6 @@ public abstract class CoprocessorHost T findCoprocessor(Class cls) {
for (E env: coprocEnvironments) {
if (cls.isAssignableFrom(env.getInstance().getClass())) {
@@ -360,7 +357,6 @@ public abstract class CoprocessorHost implements Ob
* @return An instance of ObserverContext
with the environment set
*/
@Deprecated
- @VisibleForTesting
// TODO: Remove this method, ObserverContext should not depend on RpcServer
public static ObserverContext createAndPrepare(E env) {
ObserverContextImpl ctx = new ObserverContextImpl<>(RpcServer.getRequestUser().orElse(null));
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java
index 71d8ea5fa02..2fe446277a5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java
@@ -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();
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesManager.java
index 77334db9278..e7d145036ae 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesManager.java
@@ -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 getRegionsOfFavoredNode(ServerName serverName) {
Set regionInfos = Sets.newHashSet();
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java
index c1f9a7ddacc..fccf73d37d4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java
@@ -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;
-
/**
* Wrapper for input stream(s) that takes care of the interaction of FS and HBase checksums,
* as well as closing streams. Initialization is not thread-safe, but normal operation is;
@@ -173,13 +171,11 @@ public class FSDataInputStreamWrapper implements Closeable {
}
/** For use in tests. */
- @VisibleForTesting
public FSDataInputStreamWrapper(FSDataInputStream fsdis) {
this(fsdis, fsdis);
}
/** For use in tests. */
- @VisibleForTesting
public FSDataInputStreamWrapper(FSDataInputStream fsdis, FSDataInputStream noChecksum) {
doCloseStreams = false;
stream = fsdis;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/MetricsIO.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/MetricsIO.java
index 3634ccb595f..72da73e1e92 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/MetricsIO.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/MetricsIO.java
@@ -22,8 +22,6 @@ import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
import org.apache.hadoop.hbase.regionserver.MetricsRegionServerSourceFactory;
import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
@InterfaceAudience.Private
public class MetricsIO {
@@ -40,12 +38,10 @@ public class MetricsIO {
this.wrapper = wrapper;
}
- @VisibleForTesting
public MetricsIOSource getMetricsSource() {
return source;
}
- @VisibleForTesting
public MetricsIOWrapper getWrapper() {
return wrapper;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java
index b0728d57298..f58491de044 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java
@@ -27,8 +27,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;
-
/**
* Stores all of the cache objects and configuration for a single HFile.
*/
@@ -282,7 +280,6 @@ public class CacheConfig {
* @param cacheDataOnWrite whether data blocks should be written to the cache
* when an HFile is written
*/
- @VisibleForTesting
public void setCacheDataOnWrite(boolean cacheDataOnWrite) {
this.cacheDataOnWrite = cacheDataOnWrite;
}
@@ -328,7 +325,6 @@ public class CacheConfig {
* @param evictOnClose whether blocks should be evicted from the cache when an
* HFile reader is closed
*/
- @VisibleForTesting
public void setEvictOnClose(boolean evictOnClose) {
this.evictOnClose = evictOnClose;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
index 3691635902a..92f5a47e640 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
@@ -25,9 +25,6 @@ import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.io.hfile.BlockType.BlockCategory;
import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
-
/**
* CombinedBlockCache is an abstraction layer that combines
* {@link FirstLevelBlockCache} and {@link BucketCache}. The smaller lruCache is used
@@ -379,7 +376,6 @@ public class CombinedBlockCache implements ResizableBlockCache, HeapSize {
this.l1Cache.setMaxSize(size);
}
- @VisibleForTesting
public int getRpcRefCount(BlockCacheKey cacheKey) {
return (this.l2Cache instanceof BucketCache)
? ((BucketCache) this.l2Cache).getRpcRefCount(cacheKey)
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java
index cae284a1d40..7ab4edb438a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java
@@ -204,7 +204,6 @@ public class FixedFileTrailer {
baos.writeTo(outputStream);
}
- @org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting
HFileProtos.FileTrailerProto toProtobuf() {
HFileProtos.FileTrailerProto.Builder builder = HFileProtos.FileTrailerProto.newBuilder()
.setFileInfoOffset(fileInfoOffset)
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
index 46cec4af1f8..52b6359d92c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
@@ -51,7 +51,7 @@ import org.apache.hadoop.io.Writable;
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;
/**
@@ -459,10 +459,8 @@ public final class HFile {
DataBlockEncoding getEffectiveEncodingInCache(boolean isCompaction);
- @VisibleForTesting
HFileBlock.FSReader getUncachedBlockReader();
- @VisibleForTesting
boolean prefetchComplete();
/**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
index c35150163cc..eb2557cab5a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
@@ -55,7 +55,7 @@ import org.apache.hadoop.hbase.util.ClassSize;
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;
/**
@@ -312,7 +312,6 @@ public class HFileBlock implements Cacheable {
* @param onDiskDataSizeWithHeader see {@link #onDiskDataSizeWithHeader}
* @param fileContext HFile meta data
*/
- @VisibleForTesting
public HFileBlock(BlockType blockType, int onDiskSizeWithoutHeader,
int uncompressedSizeWithoutHeader, long prevBlockOffset, ByteBuff buf, boolean fillHeader,
long offset, int nextBlockOnDiskSize, int onDiskDataSizeWithHeader, HFileContext fileContext,
@@ -525,7 +524,6 @@ public class HFileBlock implements Cacheable {
return this.allocator;
}
- @VisibleForTesting
private void sanityCheckAssertion(long valueFromBuf, long valueFromField,
String fieldName) throws IOException {
if (valueFromBuf != valueFromField) {
@@ -534,7 +532,6 @@ public class HFileBlock implements Cacheable {
}
}
- @VisibleForTesting
private void sanityCheckAssertion(BlockType valueFromBuf, BlockType valueFromField)
throws IOException {
if (valueFromBuf != valueFromField) {
@@ -551,7 +548,6 @@ public class HFileBlock implements Cacheable {
* thread-safe, because it alters the internal buffer pointer.
* Used by tests only.
*/
- @VisibleForTesting
void sanityCheck() throws IOException {
// Duplicate so no side-effects
ByteBuff dup = this.buf.duplicate().rewind();
@@ -840,7 +836,6 @@ public class HFileBlock implements Cacheable {
/**
* @param dataBlockEncoder data block encoding algorithm to use
*/
- @VisibleForTesting
public Writer(HFileDataBlockEncoder dataBlockEncoder, HFileContext fileContext) {
this(dataBlockEncoder, fileContext, ByteBuffAllocator.HEAP);
}
@@ -1403,7 +1398,6 @@ public class HFileBlock implements Cacheable {
private long fileSize;
/** The size of the header */
- @VisibleForTesting
protected final int hdrSize;
/** The filesystem used to access data */
@@ -1694,7 +1688,6 @@ public class HFileBlock implements Cacheable {
* @param intoHeap allocate the ByteBuff of block from heap or off-heap.
* @return the HFileBlock or null if there is a HBase checksum mismatch
*/
- @VisibleForTesting
protected HFileBlock readBlockDataInternal(FSDataInputStream is, long offset,
long onDiskSizeWithHeaderL, boolean pread, boolean verifyChecksum, boolean updateMetrics,
boolean intoHeap) throws IOException {
@@ -1852,7 +1845,6 @@ public class HFileBlock implements Cacheable {
}
/** An additional sanity-check in case no compression or encryption is being used. */
- @VisibleForTesting
void sanityCheckUncompressed() throws IOException {
if (onDiskSizeWithoutHeader != uncompressedSizeWithoutHeader +
totalChecksumBytes()) {
@@ -1974,7 +1966,6 @@ public class HFileBlock implements Cacheable {
return DataBlockEncoding.NONE;
}
- @VisibleForTesting
byte getChecksumType() {
return this.fileContext.getChecksumType().getCode();
}
@@ -1984,7 +1975,6 @@ public class HFileBlock implements Cacheable {
}
/** @return the size of data on disk + header. Excludes checksum. */
- @VisibleForTesting
int getOnDiskDataSizeWithHeader() {
return this.onDiskDataSizeWithHeader;
}
@@ -2023,7 +2013,6 @@ public class HFileBlock implements Cacheable {
/**
* Return the appropriate DUMMY_HEADER for the minor version
*/
- @VisibleForTesting
// TODO: Why is this in here?
byte[] getDummyHeaderForVersion() {
return getDummyHeaderForVersion(this.fileContext.isUseHBaseChecksum());
@@ -2049,7 +2038,6 @@ public class HFileBlock implements Cacheable {
* This is mostly helpful for debugging. This assumes that the block
* has minor version > 0.
*/
- @VisibleForTesting
static String toStringHeader(ByteBuff buf) throws IOException {
byte[] magicBuf = new byte[Math.min(buf.limit() - buf.position(), BlockType.MAGIC_LENGTH)];
buf.get(magicBuf);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
index 3c8823fb192..7375fa8ddb5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
@@ -52,7 +52,6 @@ import org.apache.htrace.core.TraceScope;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Implementation that can handle all hfile versions of {@link HFile.Reader}.
@@ -233,7 +232,6 @@ public abstract class HFileReaderImpl implements HFile.Reader, Configurable {
return this.hfileContext.getCellComparator();
}
- @VisibleForTesting
public Compression.Algorithm getCompressionAlgorithm() {
return trailer.getCompressionCodec();
}
@@ -1626,7 +1624,6 @@ public abstract class HFileReaderImpl implements HFile.Reader, Configurable {
* not completed, true otherwise
*/
@Override
- @VisibleForTesting
public boolean prefetchComplete() {
return PrefetchExecutor.isCompleted(path);
}
@@ -1644,7 +1641,6 @@ public abstract class HFileReaderImpl implements HFile.Reader, Configurable {
* @return Scanner on this file.
*/
@Override
- @VisibleForTesting
public HFileScanner getScanner(boolean cacheBlocks, final boolean pread) {
return getScanner(cacheBlocks, pread, false);
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
index 3f56379dc8e..4d1226b33fb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
@@ -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;
-
/**
* Common functionality needed by all versions of {@link HFile} writers.
*/
@@ -780,7 +778,6 @@ public class HFileWriterImpl implements HFile.Writer {
}
}
- @VisibleForTesting
public Cell getLastCell() {
return lastCell;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
index d15098972a7..35f4ab90cd1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
@@ -43,7 +43,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.MoreObjects;
import org.apache.hbase.thirdparty.com.google.common.base.Objects;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
@@ -641,12 +640,10 @@ public class LruBlockCache implements FirstLevelBlockCache {
}
}
- @VisibleForTesting
boolean isEvictionInProgress() {
return evictionInProgress;
}
- @VisibleForTesting
long getOverhead() {
return overhead;
}
@@ -1178,7 +1175,6 @@ public class LruBlockCache implements FirstLevelBlockCache {
}
/** Clears the cache. Used in tests. */
- @VisibleForTesting
public void clearCache() {
this.map.clear();
this.elements.set(0);
@@ -1189,7 +1185,6 @@ public class LruBlockCache implements FirstLevelBlockCache {
*
* @return the set of cached file names
*/
- @VisibleForTesting
SortedSet getCachedFileNamesForTest() {
SortedSet fileNames = new TreeSet<>();
for (BlockCacheKey cacheKey : map.keySet()) {
@@ -1198,7 +1193,6 @@ public class LruBlockCache implements FirstLevelBlockCache {
return fileNames;
}
- @VisibleForTesting
public Map getEncodingCountsForTest() {
Map counts = new EnumMap<>(DataBlockEncoding.class);
for (LruCachedBlock block : map.values()) {
@@ -1209,7 +1203,6 @@ public class LruBlockCache implements FirstLevelBlockCache {
return counts;
}
- @VisibleForTesting
Map getMapForTests() {
return map;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ReaderContextBuilder.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ReaderContextBuilder.java
index c58d5b8ce07..1f903cfbea6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ReaderContextBuilder.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ReaderContextBuilder.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
import org.apache.hadoop.hbase.io.hfile.ReaderContext.ReaderType;
import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* A builder that helps in building up the ReaderContext
@@ -82,7 +81,6 @@ public class ReaderContextBuilder {
return this;
}
- @VisibleForTesting
public ReaderContextBuilder withFileSystemAndPath(FileSystem fs, Path filePath)
throws IOException {
this.withFileSystem(fs)
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/TinyLfuBlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/TinyLfuBlockCache.java
index a90c5a33db6..a0dc30c5242 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/TinyLfuBlockCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/TinyLfuBlockCache.java
@@ -36,7 +36,6 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
import org.apache.hadoop.util.StringUtils;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.yetus.audience.InterfaceAudience;
@@ -69,7 +68,6 @@ public final class TinyLfuBlockCache implements FirstLevelBlockCache {
private BlockCache victimCache;
- @VisibleForTesting
final Cache cache;
/**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
index fca1ffabee0..de44ad7f54e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
@@ -49,7 +49,6 @@ import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.function.Consumer;
-
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.TableName;
@@ -77,7 +76,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;
@@ -113,7 +111,6 @@ public class BucketCache implements BlockCache, HeapSize {
static final String MIN_FACTOR_CONFIG_NAME = "hbase.bucketcache.minfactor";
/** Priority buckets */
- @VisibleForTesting
static final float DEFAULT_SINGLE_FACTOR = 0.25f;
static final float DEFAULT_MULTI_FACTOR = 0.50f;
static final float DEFAULT_MEMORY_FACTOR = 0.25f;
@@ -135,10 +132,8 @@ public class BucketCache implements BlockCache, HeapSize {
transient final IOEngine ioEngine;
// Store the block in this map before writing it to cache
- @VisibleForTesting
transient final RAMCache ramCache;
// In this map, store the block's meta data like offset, length
- @VisibleForTesting
transient ConcurrentHashMap backingMap;
/**
@@ -155,9 +150,7 @@ public class BucketCache implements BlockCache, HeapSize {
* WriterThread when it runs takes whatever has been recently added and 'drains' the entries
* to the BucketCache. It then updates the ramCache and backingMap accordingly.
*/
- @VisibleForTesting
transient final ArrayList> writerQueues = new ArrayList<>();
- @VisibleForTesting
transient final WriterThread[] writerThreads;
/** Volatile boolean to track if free space is in process or not */
@@ -179,7 +172,6 @@ public class BucketCache implements BlockCache, HeapSize {
* bucket cache will skip some blocks when caching. If the flag is true, we
* will wait until blocks are flushed to IOEngine.
*/
- @VisibleForTesting
boolean wait_when_cache = false;
private final BucketCacheStats cacheStats = new BucketCacheStats();
@@ -204,7 +196,6 @@ public class BucketCache implements BlockCache, HeapSize {
*
* Key set of offsets in BucketCache is limited so soft reference is the best choice here.
*/
- @VisibleForTesting
transient final IdReadWriteLock offsetLock = new IdReadWriteLock<>(ReferenceType.SOFT);
private final NavigableSet blocksByHFile = new ConcurrentSkipListSet<>((a, b) -> {
@@ -341,14 +332,12 @@ public class BucketCache implements BlockCache, HeapSize {
* Called by the constructor to start the writer threads. Used by tests that need to override
* starting the threads.
*/
- @VisibleForTesting
protected void startWriterThreads() {
for (WriterThread thread : writerThreads) {
thread.start();
}
}
- @VisibleForTesting
boolean isCacheEnabled() {
return this.cacheEnabled;
}
@@ -545,7 +534,6 @@ public class BucketCache implements BlockCache, HeapSize {
return null;
}
- @VisibleForTesting
void blockEvicted(BlockCacheKey cacheKey, BucketEntry bucketEntry, boolean decrementBlockNumber) {
bucketAllocator.freeBlock(bucketEntry.offset());
realCacheSize.add(-1 * bucketEntry.getLength());
@@ -670,7 +658,6 @@ public class BucketCache implements BlockCache, HeapSize {
return (long) Math.floor(bucketAllocator.getTotalSize() * acceptableFactor);
}
- @VisibleForTesting
long getPartitionSize(float partitionFactor) {
return (long) Math.floor(bucketAllocator.getTotalSize() * partitionFactor * minFactor);
}
@@ -865,7 +852,6 @@ public class BucketCache implements BlockCache, HeapSize {
}
// This handles flushing the RAM cache to IOEngine.
- @VisibleForTesting
class WriterThread extends Thread {
private final BlockingQueue inputQueue;
private volatile boolean writerEnabled = true;
@@ -876,7 +862,6 @@ public class BucketCache implements BlockCache, HeapSize {
}
// Used for test
- @VisibleForTesting
void disableWriter() {
this.writerEnabled = false;
}
@@ -936,7 +921,6 @@ public class BucketCache implements BlockCache, HeapSize {
* interference expected.
* @throws InterruptedException
*/
- @VisibleForTesting
void doDrain(final List entries) throws InterruptedException {
if (entries.isEmpty()) {
return;
@@ -1044,7 +1028,6 @@ public class BucketCache implements BlockCache, HeapSize {
* @param q The queue to take from.
* @return {@code receptacle} laden with elements taken from the queue or empty if none found.
*/
- @VisibleForTesting
static List getRAMQueueEntries(BlockingQueue q,
List receptacle) throws InterruptedException {
// Clear sets all entries to null and sets size to 0. We retain allocations. Presume it
@@ -1338,7 +1321,6 @@ public class BucketCache implements BlockCache, HeapSize {
/**
* Block Entry stored in the memory with key,data and so on
*/
- @VisibleForTesting
static class RAMQueueEntry {
private final BlockCacheKey key;
private final Cacheable data;
@@ -1520,7 +1502,6 @@ public class BucketCache implements BlockCache, HeapSize {
return null;
}
- @VisibleForTesting
public int getRpcRefCount(BlockCacheKey cacheKey) {
BucketEntry bucketEntry = backingMap.get(cacheKey);
if (bucketEntry != null) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java
index 2cdfc80a39c..e4a2c0b1aea 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java
@@ -27,7 +27,6 @@ import java.nio.channels.ClosedChannelException;
import java.nio.channels.FileChannel;
import java.util.Arrays;
import java.util.concurrent.locks.ReentrantLock;
-
import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException;
import org.apache.hadoop.hbase.io.hfile.Cacheable;
import org.apache.hadoop.hbase.nio.ByteBuff;
@@ -36,7 +35,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;
/**
@@ -154,7 +152,6 @@ public class FileIOEngine extends PersistentIOEngine {
return be.wrapAsCacheable(dstBuff);
}
- @VisibleForTesting
void closeFileChannels() {
for (FileChannel fileChannel: fileChannels) {
try {
@@ -283,12 +280,10 @@ public class FileIOEngine extends PersistentIOEngine {
return fileNum;
}
- @VisibleForTesting
FileChannel[] getFileChannels() {
return fileChannels;
}
- @VisibleForTesting
void refreshFileConnection(int accessFileNum, IOException ioe) throws IOException {
ReentrantLock channelLock = channelLocks[accessFileNum];
channelLock.lock();
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java
index eab2a0ec85c..f34cad5f60c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
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.protobuf.BlockingService;
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
@@ -124,7 +124,7 @@ public class NettyRpcServer extends RpcServer {
this.scheduler.init(new RpcSchedulerContext(this));
}
- @VisibleForTesting
+ @InterfaceAudience.Private
protected NettyRpcServerPreambleHandler createNettyRpcServerPreambleHandler() {
return new NettyRpcServerPreambleHandler(NettyRpcServer.this);
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerPreambleHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerPreambleHandler.java
index bac19f1bb11..855cf2fda4d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerPreambleHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerPreambleHandler.java
@@ -26,7 +26,6 @@ import org.apache.hbase.thirdparty.io.netty.channel.SimpleChannelInboundHandler;
import java.nio.ByteBuffer;
import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Handle connection preamble.
@@ -58,7 +57,6 @@ class NettyRpcServerPreambleHandler extends SimpleChannelInboundHandler
p.remove("preambleDecoder");
}
- @VisibleForTesting
protected NettyServerRpcConnection createNettyServerRpcConnection(Channel channel) {
return new NettyServerRpcConnection(rpcServer, channel);
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index 7bae06f601b..ca8593ee3d5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -33,7 +33,6 @@ import java.util.Locale;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.atomic.LongAdder;
-
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CallQueueTooBigException;
@@ -68,7 +67,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.gson.Gson;
import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService;
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
@@ -551,7 +549,6 @@ public abstract class RpcServer implements RpcServerInterface,
* @param strParam stringifiedParam to be truncated
* @return truncated trace log string
*/
- @VisibleForTesting
String truncateTraceLog(String strParam) {
if (LOG.isTraceEnabled()) {
int traceLogMaxLength = getConf().getInt(TRACE_LOG_MAX_LENGTH, DEFAULT_TRACE_LOG_MAX_LENGTH);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServerInterface.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServerInterface.java
index 99e01885b5b..e06daacf5fe 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServerInterface.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServerInterface.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.ipc;
import java.io.IOException;
import java.net.InetSocketAddress;
-
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.io.ByteBuffAllocator;
@@ -32,7 +31,6 @@ import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.security.authorize.PolicyProvider;
import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService;
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
@@ -86,7 +84,6 @@ public interface RpcServerInterface {
* Refresh authentication manager policy.
* @param pp
*/
- @VisibleForTesting
void refreshAuthManager(Configuration conf, PolicyProvider pp);
RpcScheduler getScheduler();
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CachedClusterId.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CachedClusterId.java
index 2e7baae27c3..2f75560dae8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CachedClusterId.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CachedClusterId.java
@@ -32,7 +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;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
/**
@@ -157,7 +156,6 @@ public class CachedClusterId {
return getClusterId();
}
- @VisibleForTesting
public int getCacheStats() {
return cacheMisses.get();
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 9ab43557a48..39cb5bc5459 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -213,7 +213,6 @@ 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;
@@ -222,6 +221,7 @@ import org.apache.hbase.thirdparty.org.eclipse.jetty.server.ServerConnector;
import org.apache.hbase.thirdparty.org.eclipse.jetty.servlet.ServletHolder;
import org.apache.hbase.thirdparty.org.eclipse.jetty.webapp.WebAppContext;
+import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
@@ -259,7 +259,7 @@ public class HMaster extends HRegionServer implements MasterServices {
// Tracker for meta location, if any client ZK quorum specified
private MetaLocationSyncer metaLocationSyncer;
// Tracker for active master location, if any client ZK quorum specified
- @VisibleForTesting
+ @InterfaceAudience.Private
MasterAddressSyncer masterAddressSyncer;
// Tracker for auto snapshot cleanup state
SnapshotCleanupTracker snapshotCleanupTracker;
@@ -612,7 +612,7 @@ public class HMaster extends HRegionServer implements MasterServices {
}
}
- @VisibleForTesting
+ @InterfaceAudience.Private
public MasterRpcServices getMasterRpcServices() {
return (MasterRpcServices)rpcServices;
}
@@ -669,7 +669,7 @@ public class HMaster extends HRegionServer implements MasterServices {
* Initialize all ZK based system trackers. But do not include {@link RegionServerTracker}, it
* should have already been initialized along with {@link ServerManager}.
*/
- @VisibleForTesting
+ @InterfaceAudience.Private
protected void initializeZKBasedSystemTrackers()
throws IOException, InterruptedException, KeeperException, ReplicationException {
this.balancer = LoadBalancerFactory.getLoadBalancer(conf);
@@ -729,7 +729,7 @@ public class HMaster extends HRegionServer implements MasterServices {
}
// Will be overriden in test to inject customized AssignmentManager
- @VisibleForTesting
+ @InterfaceAudience.Private
protected AssignmentManager createAssignmentManager(MasterServices master) {
return new AssignmentManager(master);
}
@@ -1104,7 +1104,7 @@ public class HMaster extends HRegionServer implements MasterServices {
* @return True if meta is UP and online and startup can progress. Otherwise, meta is not online
* and we will hold here until operator intervention.
*/
- @VisibleForTesting
+ @InterfaceAudience.Private
public boolean waitForMetaOnline() {
return isRegionOnline(RegionInfoBuilder.FIRST_META_REGIONINFO);
}
@@ -1145,7 +1145,7 @@ public class HMaster extends HRegionServer implements MasterServices {
* (TODO: Fix this! NS should not hold-up startup).
* @return True if namespace table is up/online.
*/
- @VisibleForTesting
+ @InterfaceAudience.Private
public boolean waitForNamespaceOnline() {
List ris = this.assignmentManager.getRegionStates().
getRegionsOfTable(TableName.NAMESPACE_TABLE_NAME);
@@ -1167,7 +1167,7 @@ public class HMaster extends HRegionServer implements MasterServices {
* Adds the {@code MasterQuotasObserver} to the list of configured Master observers to
* automatically remove quotas for a table when that table is deleted.
*/
- @VisibleForTesting
+ @InterfaceAudience.Private
public void updateConfigurationForQuotasObserver(Configuration conf) {
// We're configured to not delete quotas on table deletion, so we don't need to add the obs.
if (!conf.getBoolean(
@@ -1204,7 +1204,7 @@ public class HMaster extends HRegionServer implements MasterServices {
* Will be overridden in tests.
*
*/
- @VisibleForTesting
+ @InterfaceAudience.Private
protected ServerManager createServerManager(final MasterServices master) throws IOException {
// We put this out here in a method so can do a Mockito.spy and stub it out
// w/ a mocked up ServerManager.
@@ -1218,7 +1218,7 @@ public class HMaster extends HRegionServer implements MasterServices {
}
// Will be overridden in tests
- @VisibleForTesting
+ @InterfaceAudience.Private
protected void initClusterSchemaService() throws IOException, InterruptedException {
this.clusterSchemaService = new ClusterSchemaServiceImpl(this);
this.clusterSchemaService.startAsync();
@@ -1888,7 +1888,7 @@ public class HMaster extends HRegionServer implements MasterServices {
// Public so can be accessed by tests. Blocks until move is done.
// Replace with an async implementation from which you can get
// a success/failure result.
- @VisibleForTesting
+ @InterfaceAudience.Private
public void move(final byte[] encodedRegionName, byte[] destServerName) throws HBaseIOException {
RegionState regionState = assignmentManager.getRegionStates().
getRegionState(Bytes.toString(encodedRegionName));
@@ -2716,7 +2716,7 @@ public class HMaster extends HRegionServer implements MasterServices {
}
}
- @VisibleForTesting
+ @InterfaceAudience.Private
protected void checkServiceStarted() throws ServerNotRunningYetException {
if (!serviceStarted) {
throw new ServerNotRunningYetException("Server is not running yet");
@@ -2771,7 +2771,7 @@ public class HMaster extends HRegionServer implements MasterServices {
return maintenanceMode;
}
- @VisibleForTesting
+ @InterfaceAudience.Private
public void setInitialized(boolean isInitialized) {
procedureExecutor.getEnvironment().setEventReady(initialized, isInitialized);
}
@@ -3617,7 +3617,7 @@ public class HMaster extends HRegionServer implements MasterServices {
/**
* This method modifies the master's configuration in order to inject replication-related features
*/
- @VisibleForTesting
+ @InterfaceAudience.Private
public static void decorateMasterConfiguration(Configuration conf) {
String plugins = conf.get(HBASE_MASTER_LOGCLEANER_PLUGINS);
String cleanerClass = ReplicationLogCleaner.class.getCanonicalName();
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
index 31d365a9dcb..51efc0645a9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
@@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.log.HBaseMarkers;
import org.apache.hadoop.hbase.mob.MobConstants;
-import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.security.access.SnapshotScannerHDFSAclHelper;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CommonFSUtils;
@@ -43,8 +42,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 abstracts a bunch of operations the HMaster needs to interact with
* the underlying file system like creating the initial layout, checking file
@@ -297,7 +294,6 @@ public class MasterFileSystem {
* Make sure the hbase temp directory exists and is empty.
* NOTE that this method is only executed once just after the master becomes the active one.
*/
- @VisibleForTesting
void checkTempDir(final Path tmpdir, final Configuration c, final FileSystem fs)
throws IOException {
// If the temp directory exists, clear the content (left over, from the previous run)
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index 9dd10becea9..3543ed99e25 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -21,7 +21,6 @@ import com.google.protobuf.Service;
import java.io.IOException;
import java.util.List;
-
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableDescriptors;
@@ -55,7 +54,6 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
import org.apache.hadoop.hbase.security.access.AccessChecker;
import org.apache.hadoop.hbase.security.access.ZKPermissionWatcher;
import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* A curated subset of services provided by {@link HMaster}.
@@ -138,7 +136,6 @@ public interface MasterServices extends Server {
/**
* @return Tripped when Master has finished initialization.
*/
- @VisibleForTesting
public ProcedureEvent> getInitializedEvent();
/**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java
index fe9107f222d..25b661c69ce 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java
@@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.wal.WALSplitter;
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 abstracts a bunch of operations the HMaster needs
@@ -66,7 +65,6 @@ public class MasterWalManager {
/**
* Filter *out* WAL files that are for the hbase:meta Region; i.e. return user-space WALs only.
*/
- @VisibleForTesting
public final static PathFilter NON_META_FILTER = new PathFilter() {
@Override
public boolean accept(Path p) {
@@ -123,7 +121,6 @@ public class MasterWalManager {
}
}
- @VisibleForTesting
SplitLogManager getSplitLogManager() {
return this.splitLogManager;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
index ded766adc01..16cef3e3eaf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
@@ -62,7 +62,7 @@ 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.protobuf.ByteString;
import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -250,7 +250,6 @@ public class ServerManager {
}
}
- @VisibleForTesting
public void regionServerReport(ServerName sn,
ServerMetrics sl) throws YouAreDeadException {
checkIsDead(sn, "REPORT");
@@ -402,7 +401,6 @@ public class ServerManager {
* Adds the onlineServers list. onlineServers should be locked.
* @param serverName The remote servers name.
*/
- @VisibleForTesting
void recordNewServerWithLock(final ServerName serverName, final ServerMetrics sl) {
LOG.info("Registering regionserver=" + serverName);
this.onlineServers.put(serverName, sl);
@@ -540,7 +538,7 @@ public class ServerManager {
* going down or we already have queued an SCP for this server or SCP processing is
* currently disabled because we are in startup phase).
*/
- @VisibleForTesting // Redo test so we can make this protected.
+ // Redo test so we can make this protected.
public synchronized long expireServer(final ServerName serverName) {
return expireServer(serverName, false);
@@ -595,7 +593,6 @@ public class ServerManager {
* Called when server has expired.
*/
// Locking in this class needs cleanup.
- @VisibleForTesting
public synchronized void moveFromOnlineToDeadServers(final ServerName sn) {
synchronized (this.onlineServers) {
boolean online = this.onlineServers.containsKey(sn);
@@ -985,7 +982,6 @@ public class ServerManager {
flushedSequenceIdByRegion.remove(encodedName);
}
- @VisibleForTesting
public boolean isRegionInServerManagerStates(final RegionInfo hri) {
final byte[] encodedName = hri.getEncodedNameAsBytes();
return (storeFlushedSequenceIdsByRegion.containsKey(encodedName)
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
index 3e0c7460eaf..465a59367ca 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
@@ -55,7 +55,6 @@ import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Distributes the task of log splitting to the available region servers.
@@ -104,7 +103,6 @@ public class SplitLogManager {
private long unassignedTimeout;
private long lastTaskCreateTime = Long.MAX_VALUE;
- @VisibleForTesting
final ConcurrentMap tasks = new ConcurrentHashMap<>();
private TimeoutMonitor timeoutMonitor;
@@ -165,7 +163,6 @@ public class SplitLogManager {
* {@link org.apache.hadoop.hbase.wal.WALSplitter#split(Path, Path, Path, FileSystem,
* Configuration, org.apache.hadoop.hbase.wal.WALFactory)} for tests.
*/
- @VisibleForTesting
public static FileStatus[] getFileList(final Configuration conf, final List logDirs,
final PathFilter filter)
throws IOException {
@@ -375,7 +372,6 @@ public class SplitLogManager {
}
}
- @VisibleForTesting
ConcurrentMap getTasks() {
return tasks;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitWALManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitWALManager.java
index 9ff84dc942e..48c19c24ed0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitWALManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitWALManager.java
@@ -43,11 +43,10 @@ import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.hadoop.hbase.wal.WALSplitUtil;
+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;
/**
* Create {@link SplitWALProcedure} for each WAL which need to split. Manage the workers for each
@@ -145,7 +144,6 @@ public class SplitWALManager {
return !fs.exists(new Path(rootDir, walPath));
}
- @VisibleForTesting
List createSplitWALProcedures(List splittingWALs,
ServerName crashedServer) {
return splittingWALs.stream()
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java
index 25c94ba5bbc..fe304cebf06 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java
@@ -27,14 +27,11 @@ import org.apache.hadoop.hbase.procedure2.ProcedureMetrics;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
-import org.apache.yetus.audience.InterfaceAudience;
-
-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.MasterProcedureProtos.AssignRegionStateData;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionTransitionState;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* Leave here only for checking if we can successfully start the master.
@@ -143,7 +140,6 @@ public class AssignProcedure extends RegionTransitionProcedure {
return env.getAssignmentManager().getAssignmentManagerMetrics().getAssignProcMetrics();
}
- @VisibleForTesting
@Override
public void setProcId(long procId) {
super.setProcId(procId);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
index 47b81f5d269..6e5dccb5777 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
@@ -83,7 +83,7 @@ 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.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
@@ -177,7 +177,6 @@ public class AssignmentManager {
this(master, new RegionStateStore(master));
}
- @VisibleForTesting
AssignmentManager(final MasterServices master, final RegionStateStore stateStore) {
this.master = master;
this.regionStateStore = stateStore;
@@ -504,7 +503,6 @@ public class AssignmentManager {
* This is a bit dirty, should be reconsidered after we decide whether to keep the
* {@link #processOfflineRegions()} method.
*/
- @VisibleForTesting
public void wakeMetaLoadedEvent() {
metaLoadEvent.wake(getProcedureScheduler());
assert isMetaLoaded() : "expected meta to be loaded";
@@ -774,7 +772,6 @@ public class AssignmentManager {
return createRoundRobinAssignProcedures(hris, null);
}
- @VisibleForTesting
static int compare(TransitRegionStateProcedure left, TransitRegionStateProcedure right) {
if (left.getRegion().isMetaRegion()) {
if (right.getRegion().isMetaRegion()) {
@@ -1359,7 +1356,6 @@ public class AssignmentManager {
private int totalRITsTwiceThreshold = 0;
private int totalRITs = 0;
- @VisibleForTesting
public RegionInTransitionStat(final Configuration conf) {
this.ritThreshold =
conf.getInt(METRICS_RIT_STUCK_WARNING_THRESHOLD, DEFAULT_RIT_STUCK_WARNING_THRESHOLD);
@@ -2255,7 +2251,6 @@ public class AssignmentManager {
.collect(Collectors.toList());
}
- @VisibleForTesting
MasterServices getMaster() {
return master;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
index 7b43b5cc635..0c5db0dc9b3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
@@ -60,7 +60,7 @@ import org.apache.hadoop.hbase.wal.WALSplitUtil;
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;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
@@ -753,7 +753,6 @@ public class MergeTableRegionsProcedure
/**
* @return The merged region. Maybe be null if called to early or we failed.
*/
- @VisibleForTesting
RegionInfo getMergedRegion() {
return this.mergedRegion;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateNode.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateNode.java
index af2c9fb85b9..98c9ebae4c1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateNode.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateNode.java
@@ -36,8 +36,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;
-
/**
* Current Region State. Most fields are synchronized with meta region, i.e, we will update meta
* immediately after we modify this RegionStateNode, and usually under the lock. The only exception
@@ -77,7 +75,6 @@ public class RegionStateNode implements Comparable {
}
}
- @VisibleForTesting
final Lock lock = new ReentrantLock();
private final RegionInfo regionInfo;
private final ProcedureEvent> event;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
index 54765402fbf..d3553f11a32 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
@@ -45,8 +45,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;
-
/**
* RegionStates contains a set of Maps that describes the in-memory state of the AM, with
* the regions available in the system, the region in transition, the offline regions and
@@ -105,7 +103,6 @@ public class RegionStates {
serverMap.clear();
}
- @VisibleForTesting
public boolean isRegionInRegionStates(final RegionInfo hri) {
return (regionsMap.containsKey(hri.getRegionName()) || regionInTransition.containsKey(hri)
|| regionOffline.containsKey(hri));
@@ -114,7 +111,6 @@ public class RegionStates {
// ==========================================================================
// RegionStateNode helpers
// ==========================================================================
- @VisibleForTesting
RegionStateNode createRegionStateNode(RegionInfo regionInfo) {
RegionStateNode newNode = new RegionStateNode(regionInfo, regionInTransition);
RegionStateNode oldNode = regionsMap.putIfAbsent(regionInfo.getRegionName(), newNode);
@@ -758,7 +754,6 @@ public class RegionStates {
/**
* @return Pertinent ServerStateNode or NULL if none found (Do not make modifications).
*/
- @VisibleForTesting
public ServerStateNode getServerNode(final ServerName serverName) {
return serverMap.get(serverName);
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
index 2e767718367..bc46e19978d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
@@ -31,10 +31,9 @@ import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure;
import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionTransitionState;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* Leave here only for checking if we can successfully start the master.
@@ -62,12 +61,10 @@ public abstract class RegionTransitionProcedure extends ProcedureThis is a best effort load balancer. Given a Cost function F(C) => x It will
* randomly try and mutate the cluster to Cprime. If F(Cprime) < F(C) then the
@@ -363,7 +360,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
return !balanced;
}
- @VisibleForTesting
+ @InterfaceAudience.Private
Cluster.Action nextAction(Cluster cluster) {
return candidateGenerators.get(RANDOM.nextInt(candidateGenerators.size()))
.generate(cluster);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
index 4331d490c28..9416e5a12a6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
@@ -28,7 +28,6 @@ import java.util.Map;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.stream.Collectors;
-
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@@ -42,7 +41,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.collect.ImmutableSet;
import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
@@ -373,7 +371,6 @@ public abstract class CleanerChore extends Schedu
}
}
- @VisibleForTesting
int getChorePoolSize() {
return pool.getSize();
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
index 6926f12c49e..ff288572e25 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
@@ -39,7 +39,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 Chore, every time it runs, will clear the HFiles in the hfile archive
* folder that are deletable for each HFile cleaner in the chain.
@@ -82,12 +81,10 @@ public class HFileCleaner extends CleanerChore
public static final String HFILE_DELETE_THREAD_TIMEOUT_MSEC =
"hbase.regionserver.hfilecleaner.thread.timeout.msec";
- @VisibleForTesting
static final long DEFAULT_HFILE_DELETE_THREAD_TIMEOUT_MSEC = 60 * 1000L;
public static final String HFILE_DELETE_THREAD_CHECK_INTERVAL_MSEC =
"hbase.regionserver.hfilecleaner.thread.check.interval.msec";
- @VisibleForTesting
static final long DEFAULT_HFILE_DELETE_THREAD_CHECK_INTERVAL_MSEC = 1000L;
private static final Logger LOG = LoggerFactory.getLogger(HFileCleaner.class);
@@ -383,42 +380,34 @@ public class HFileCleaner extends CleanerChore
}
}
- @VisibleForTesting
public List getCleanerThreads() {
return threads;
}
- @VisibleForTesting
public long getNumOfDeletedLargeFiles() {
return deletedLargeFiles.get();
}
- @VisibleForTesting
public long getNumOfDeletedSmallFiles() {
return deletedSmallFiles.get();
}
- @VisibleForTesting
public long getLargeQueueInitSize() {
return largeQueueInitSize;
}
- @VisibleForTesting
public long getSmallQueueInitSize() {
return smallQueueInitSize;
}
- @VisibleForTesting
public long getThrottlePoint() {
return throttlePoint;
}
- @VisibleForTesting
long getCleanerThreadTimeoutMsec() {
return cleanerThreadTimeoutMsec;
}
- @VisibleForTesting
long getCleanerThreadCheckIntervalMsec() {
return cleanerThreadCheckIntervalMsec;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java
index 5fa115c9b8b..f65713ebf26 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java
@@ -27,7 +27,6 @@ import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.stream.Collectors;
-
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@@ -41,7 +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;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
/**
@@ -59,7 +57,6 @@ public class LogCleaner extends CleanerChore
public static final String OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC =
"hbase.oldwals.cleaner.thread.timeout.msec";
- @VisibleForTesting
static final long DEFAULT_OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC = 60 * 1000L;
private final LinkedBlockingQueue pendingDelete;
@@ -138,12 +135,10 @@ public class LogCleaner extends CleanerChore
interruptOldWALsCleaner();
}
- @VisibleForTesting
int getSizeOfCleaners() {
return oldWALsCleaner.size();
}
- @VisibleForTesting
long getCleanerThreadTimeoutMsec() {
return cleanerThreadTimeoutMsec;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/CatalogJanitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/CatalogJanitor.java
index 330ea01d42f..0a23db68ca0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/CatalogJanitor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/CatalogJanitor.java
@@ -59,8 +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;
-
/**
* A janitor for the catalog tables. Scans the hbase:meta
catalog table on a period.
* Makes a lastReport on state of hbase:meta. Looks for unused regions to garbage collect. Scan of
@@ -217,7 +215,6 @@ public class CatalogJanitor extends ScheduledChore {
* @return Return generated {@link Report}
*/
// will be override in tests.
- @VisibleForTesting
protected Report scanForReport() throws IOException {
ReportMakingVisitor visitor = new ReportMakingVisitor(this.services);
// Null tablename means scan all of meta.
@@ -303,7 +300,6 @@ public class CatalogJanitor extends ScheduledChore {
}
}
- @VisibleForTesting
static boolean cleanParent(MasterServices services, RegionInfo parent, Result rowContent)
throws IOException {
// Check whether it is a merged region and if it is clean of references.
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/MetaFixer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/MetaFixer.java
index 791e3d7d5ac..dee8f545d6c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/MetaFixer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/MetaFixer.java
@@ -48,11 +48,9 @@ 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;
-
/**
* Server-side fixing of bad or inconsistent state in hbase:meta.
* Distinct from MetaTableAccessor because {@link MetaTableAccessor} is about low-level
@@ -261,7 +259,6 @@ public class MetaFixer {
* @param maxMergeCount Maximum regions to merge at a time (avoid merging
* 100k regions in one go!)
*/
- @VisibleForTesting
static List> calculateMerges(int maxMergeCount,
List> overlaps) {
if (overlaps.isEmpty()) {
@@ -333,7 +330,6 @@ public class MetaFixer {
* @return Either a
or b
, whichever has the
* endkey that is furthest along in the Table.
*/
- @VisibleForTesting
static RegionInfo getRegionInfoWithLargestEndKey(RegionInfo a, RegionInfo b) {
if (a == null) {
// b may be null.
@@ -361,7 +357,6 @@ public class MetaFixer {
* @return True if an overlap found between passed in ri
and
* the pair
. Does NOT check the pairs themselves overlap.
*/
- @VisibleForTesting
static boolean isOverlap(RegionInfo ri, Pair pair) {
if (ri == null || pair == null) {
// Can't be an overlap in either of these cases.
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockManager.java
index 26b838dcefb..aaf51526f4f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockManager.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.util.NonceKey;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Functions to acquire lock on table/namespace/regions.
@@ -192,7 +191,6 @@ public final class LockManager {
return "MasterLock: proc = " + proc.toString();
}
- @VisibleForTesting
LockProcedure getProc() {
return proc;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
index 0e6d61dd75c..4242ed662d0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
@@ -44,7 +44,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.Lists;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -401,7 +400,6 @@ public class CreateTableProcedure
return !getTableName().isSystemTable();
}
- @VisibleForTesting
RegionInfo getFirstRegionInfo() {
if (newRegions == null || newRegions.isEmpty()) {
return null;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
index 80db5ef53bc..007d3427374 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
@@ -43,8 +43,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;
-
/**
* ProcedureScheduler for the Master Procedures.
* This ProcedureScheduler tries to provide to the ProcedureExecutor procedures
@@ -660,7 +658,6 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
* @return true if deletion succeeded, false otherwise meaning that there are
* other new operations pending for that table (e.g. a new create).
*/
- @VisibleForTesting
boolean markTableAsDeleted(final TableName table, final Procedure> procedure) {
schedLock();
try {
@@ -1019,7 +1016,6 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
/**
* For debugging. Expensive.
*/
- @VisibleForTesting
public String dumpLocks() throws IOException {
schedLock();
try {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
index 3b4ee4cb20f..17e5522efd7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
@@ -35,15 +35,15 @@ import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher;
import org.apache.hadoop.hbase.regionserver.RegionServerAbortedException;
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.RetryCounter;
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.ArrayListMultimap;
import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
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.AdminProtos.AdminService;
@@ -383,7 +383,6 @@ public class RSProcedureDispatcher
}
// will be overridden in test.
- @VisibleForTesting
protected ExecuteProceduresResponse sendRequest(final ServerName serverName,
final ExecuteProceduresRequest request) throws IOException {
try {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitWALProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitWALProcedure.java
index 4ae408f417d..88996cdfd41 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitWALProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitWALProcedure.java
@@ -16,6 +16,7 @@
* limitations under the License.
*/
package org.apache.hadoop.hbase.master.procedure;
+
import java.io.IOException;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.ServerName;
@@ -30,7 +31,7 @@ import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
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;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
@@ -163,7 +164,6 @@ public class SplitWALProcedure
return walPath;
}
- @VisibleForTesting
public ServerName getWorker(){
return worker;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
index 290e0c19df1..26eceb90745 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
@@ -22,7 +22,6 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
-
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException;
@@ -37,7 +36,7 @@ import org.apache.hadoop.hbase.util.ModifyRegionUtils;
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;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
@@ -318,7 +317,6 @@ public class TruncateTableProcedure
}
}
- @VisibleForTesting
RegionInfo getFirstRegionInfo() {
if (regions == null || regions.isEmpty()) {
return null;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/region/MasterRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/region/MasterRegion.java
index 82d013ca561..c9666378b83 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/region/MasterRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/region/MasterRegion.java
@@ -50,7 +50,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.math.IntMath;
/**
@@ -96,10 +95,8 @@ public final class MasterRegion {
private final WALFactory walFactory;
- @VisibleForTesting
final HRegion region;
- @VisibleForTesting
final MasterRegionFlusherAndCompactor flusherAndCompactor;
private MasterRegionWALRoller walRoller;
@@ -141,17 +138,14 @@ public final class MasterRegion {
return region.getScanner(scan);
}
- @VisibleForTesting
public FlushResult flush(boolean force) throws IOException {
return region.flush(force);
}
- @VisibleForTesting
public void requestRollAll() {
walRoller.requestRollAll();
}
- @VisibleForTesting
public void waitUntilWalRollFinished() throws InterruptedException {
walRoller.waitUntilWalRollFinished();
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
index 8f576e0d492..334cfb44c30 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
@@ -44,8 +44,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.MasterProcedureProtos.PeerModificationState;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
@@ -157,7 +155,6 @@ public abstract class ModifyPeerProcedure extends AbstractPeerProcedure getSnapshotsInProgress() throws IOException {
List snapshotInProgress = Lists.newArrayList();
// only add those files to the cache, but not to the known snapshots
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
index 46eab4bbd05..83025d9c641 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
@@ -92,7 +92,6 @@ 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.util.concurrent.ThreadFactoryBuilder;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -196,7 +195,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
* @param coordinator procedure coordinator instance. exposed for testing.
* @param pool HBase ExecutorServcie instance, exposed for testing.
*/
- @VisibleForTesting
+ @InterfaceAudience.Private
SnapshotManager(final MasterServices master, ProcedureCoordinator coordinator,
ExecutorService pool, int sentinelCleanInterval)
throws IOException, UnsupportedOperationException {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
index ed299028e4e..fc323eaf371 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
@@ -82,7 +82,6 @@ import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
import org.apache.hadoop.hbase.regionserver.StoreScanner;
import org.apache.hadoop.hbase.security.EncryptionUtil;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -404,7 +403,6 @@ public class PartitionedMobCompactor extends MobCompactor {
}
}
- @VisibleForTesting
List getListOfDelFilesForPartition(final CompactionPartition partition,
final List delPartitions) {
// Binary search for startKey and endKey
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ProcedureCoordinator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ProcedureCoordinator.java
index 48e96cd3bdd..1ea0df06e77 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ProcedureCoordinator.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ProcedureCoordinator.java
@@ -35,7 +35,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.MapMaker;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
@@ -70,7 +69,7 @@ public class ProcedureCoordinator {
*
* @param pool Used for executing procedures.
*/
- @VisibleForTesting // Only used in tests. SimpleMasterProcedureManager is a test class.
+ // Only used in tests. SimpleMasterProcedureManager is a test class.
public ProcedureCoordinator(ProcedureCoordinatorRpcs rpcs, ThreadPoolExecutor pool) {
this(rpcs, pool, TIMEOUT_MILLIS_DEFAULT, WAKE_MILLIS_DEFAULT);
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/store/region/RegionProcedureStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/store/region/RegionProcedureStore.java
index c7bdc9a8acc..5e2931d88ac 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/store/region/RegionProcedureStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/store/region/RegionProcedureStore.java
@@ -63,7 +63,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.ImmutableSet;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
@@ -87,7 +86,6 @@ public class RegionProcedureStore extends ProcedureStoreBase {
private final LeaseRecovery leaseRecovery;
- @VisibleForTesting
final MasterRegion region;
private int numThreads;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/AverageIntervalRateLimiter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/AverageIntervalRateLimiter.java
index b2457959190..d10e6eacc7c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/AverageIntervalRateLimiter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/AverageIntervalRateLimiter.java
@@ -13,7 +13,6 @@ package org.apache.hadoop.hbase.quotas;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* This limiter will refill resources at every TimeUnit/resources interval. For example: For a
@@ -62,13 +61,11 @@ public class AverageIntervalRateLimiter extends RateLimiter {
}
// This method is for strictly testing purpose only
- @VisibleForTesting
@Override
public void setNextRefillTime(long nextRefillTime) {
this.nextRefillTime = nextRefillTime;
}
- @VisibleForTesting
@Override
public long getNextRefillTime() {
return this.nextRefillTime;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FileArchiverNotifierFactoryImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FileArchiverNotifierFactoryImpl.java
index 36b535616ae..e4ba7a49891 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FileArchiverNotifierFactoryImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FileArchiverNotifierFactoryImpl.java
@@ -26,8 +26,6 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
/**
* A factory for getting instances of {@link FileArchiverNotifier}.
*/
@@ -46,12 +44,10 @@ public final class FileArchiverNotifierFactoryImpl implements FileArchiverNotifi
return CURRENT_INSTANCE;
}
- @VisibleForTesting
static void setInstance(FileArchiverNotifierFactory inst) {
CURRENT_INSTANCE = Objects.requireNonNull(inst);
}
- @VisibleForTesting
static void reset() {
CURRENT_INSTANCE = DEFAULT_INSTANCE;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FileSystemUtilizationChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FileSystemUtilizationChore.java
index edda4dfeafd..501ad8b8b2e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FileSystemUtilizationChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FileSystemUtilizationChore.java
@@ -161,7 +161,7 @@ public class FileSystemUtilizationChore extends ScheduledChore {
return regionSize;
}
- // VisibleForTesting
+ // visible for testing
RegionSizeStore getRegionSizeStore() {
return rs.getRegionServerSpaceQuotaManager().getRegionSizeStore();
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FixedIntervalRateLimiter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FixedIntervalRateLimiter.java
index e67eda5c5bd..50f40afbe3c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FixedIntervalRateLimiter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FixedIntervalRateLimiter.java
@@ -13,7 +13,6 @@ package org.apache.hadoop.hbase.quotas;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* With this limiter resources will be refilled only after a fixed interval of time.
@@ -44,13 +43,11 @@ public class FixedIntervalRateLimiter extends RateLimiter {
}
// This method is for strictly testing purpose only
- @VisibleForTesting
@Override
public void setNextRefillTime(long nextRefillTime) {
this.nextRefillTime = nextRefillTime;
}
- @VisibleForTesting
@Override
public long getNextRefillTime() {
return this.nextRefillTime;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
index 8d8cf569edd..99edae834dc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
@@ -48,7 +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.collect.HashMultimap;
import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
@@ -676,7 +675,6 @@ public class MasterQuotaManager implements RegionStateListener {
}
}
- @VisibleForTesting
void initializeRegionSizes() {
assert regionSizes == null;
this.regionSizes = new ConcurrentHashMap<>();
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaCache.java
index ce26366b6f9..92267aeacb3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaCache.java
@@ -20,11 +20,6 @@ package org.apache.hadoop.hbase.quotas;
import static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsent;
-import org.apache.hadoop.hbase.ClusterMetrics.Option;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
import java.io.IOException;
import java.util.ArrayList;
import java.util.EnumSet;
@@ -34,10 +29,13 @@ import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
+import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.ScheduledChore;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.security.UserGroupInformation;
@@ -179,32 +177,26 @@ public class QuotaCache implements Stoppable {
return computeIfAbsent(quotasMap, key, QuotaState::new, this::triggerCacheRefresh);
}
- @VisibleForTesting
void triggerCacheRefresh() {
refreshChore.triggerNow();
}
- @VisibleForTesting
long getLastUpdate() {
return refreshChore.lastUpdate;
}
- @VisibleForTesting
Map getNamespaceQuotaCache() {
return namespaceQuotaCache;
}
- @VisibleForTesting
Map getRegionServerQuotaCache() {
return regionServerQuotaCache;
}
- @VisibleForTesting
Map getTableQuotaCache() {
return tableQuotaCache;
}
- @VisibleForTesting
Map getUserQuotaCache() {
return userQuotaCache;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaObserverChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaObserverChore.java
index 425c24013aa..0a8bfe18abb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaObserverChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaObserverChore.java
@@ -25,7 +25,6 @@ import java.util.Objects;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
-
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ScheduledChore;
import org.apache.hadoop.hbase.Stoppable;
@@ -41,10 +40,11 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
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.HashMultimap;
import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
import org.apache.hbase.thirdparty.com.google.common.collect.Multimap;
+
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
/**
@@ -521,12 +521,10 @@ public class QuotaObserverChore extends ScheduledChore {
}
}
- @VisibleForTesting
QuotaSnapshotStore getTableSnapshotStore() {
return tableSnapshotStore;
}
- @VisibleForTesting
QuotaSnapshotStore getNamespaceSnapshotStore() {
return namespaceSnapshotStore;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RateLimiter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RateLimiter.java
index 852d8a68f02..a29b90d5217 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RateLimiter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RateLimiter.java
@@ -23,8 +23,6 @@ import java.util.concurrent.TimeUnit;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
/**
* Simple rate limiter.
*
@@ -230,9 +228,8 @@ public abstract class RateLimiter {
}
// These two method are for strictly testing purpose only
- @VisibleForTesting
+
public abstract void setNextRefillTime(long nextRefillTime);
- @VisibleForTesting
public abstract long getNextRefillTime();
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerRpcQuotaManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerRpcQuotaManager.java
index 0f96de53565..9a27160b1f6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerRpcQuotaManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerRpcQuotaManager.java
@@ -29,11 +29,12 @@ import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
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.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
/**
@@ -83,7 +84,6 @@ public class RegionServerRpcQuotaManager {
}
}
- @VisibleForTesting
protected boolean isRpcThrottleEnabled() {
return rpcThrottleEnabled;
}
@@ -108,7 +108,6 @@ public class RegionServerRpcQuotaManager {
}
}
- @VisibleForTesting
QuotaCache getQuotaCache() {
return quotaCache;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerSpaceQuotaManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerSpaceQuotaManager.java
index 39727000ffd..07c6a197566 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerSpaceQuotaManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerSpaceQuotaManager.java
@@ -24,16 +24,14 @@ import java.util.Objects;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicReference;
import java.util.Map.Entry;
-
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.Connection;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+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.protobuf.TextFormat;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -68,7 +66,6 @@ public class RegionServerSpaceQuotaManager {
this(rsServices, SpaceViolationPolicyEnforcementFactory.getInstance());
}
- @VisibleForTesting
RegionServerSpaceQuotaManager(
RegionServerServices rsServices, SpaceViolationPolicyEnforcementFactory factory) {
this.rsServices = Objects.requireNonNull(rsServices);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java
index 60cc92f428c..556cf04c6c3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java
@@ -18,7 +18,6 @@
*/
package org.apache.hadoop.hbase.regionserver;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import java.io.IOException;
import java.util.List;
import java.util.NavigableSet;
@@ -377,12 +376,10 @@ public abstract class AbstractMemStore implements MemStore {
return comparator;
}
- @VisibleForTesting
MutableSegment getActive() {
return active;
}
- @VisibleForTesting
ImmutableSegment getSnapshot() {
return snapshot;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AnnotationReadingPriorityFunction.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AnnotationReadingPriorityFunction.java
index 2dbc1066be7..df6295702a3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AnnotationReadingPriorityFunction.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AnnotationReadingPriorityFunction.java
@@ -20,14 +20,18 @@ package org.apache.hadoop.hbase.regionserver;
import java.lang.reflect.Method;
import java.util.HashMap;
import java.util.Map;
-
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ipc.PriorityFunction;
+import org.apache.hadoop.hbase.ipc.QosPriority;
+import org.apache.hadoop.hbase.security.User;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.ipc.PriorityFunction;
-import org.apache.hadoop.hbase.ipc.QosPriority;
+
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
+
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest;
@@ -39,11 +43,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanReques
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-import org.apache.hbase.thirdparty.com.google.protobuf.Message;
-import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
-import org.apache.hadoop.hbase.security.User;
-
/**
* Reads special method annotations and table names to figure a priority for use by QoS facility in
* ipc; e.g: rpcs to hbase:meta get priority.
@@ -275,7 +274,6 @@ public class AnnotationReadingPriorityFunction implements PriorityFunction {
return 0;
}
- @VisibleForTesting
void setRegionServer(final HRegionServer hrs) {
this.rpcServices = hrs.getRSRpcServices();
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellSet.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellSet.java
index d74655d1b7a..e1cfd15ed19 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellSet.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellSet.java
@@ -18,8 +18,6 @@
*/
package org.apache.hadoop.hbase.regionserver;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import java.util.Collection;
import java.util.Comparator;
import java.util.Iterator;
@@ -30,6 +28,7 @@ import java.util.concurrent.ConcurrentSkipListMap;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.HConstants;
import org.apache.yetus.audience.InterfaceAudience;
/**
@@ -62,13 +61,11 @@ public class CellSet implements NavigableSet {
this.numUniqueKeys = numUniqueKeys;
}
- @VisibleForTesting
CellSet(final NavigableMap m) {
this.delegatee = m;
this.numUniqueKeys = UNKNOWN_NUM_UNIQUES;
}
- @VisibleForTesting
NavigableMap getDelegatee() {
return delegatee;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java
index 136efeec201..1023890dac2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java
@@ -19,11 +19,9 @@ package org.apache.hadoop.hbase.regionserver;
import java.nio.ByteBuffer;
import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes;
+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;
/**
@@ -176,7 +174,6 @@ public abstract class Chunk {
+ (data.capacity() - nextFreeOffset.get());
}
- @VisibleForTesting
int getNextFreeOffset() {
return this.nextFreeOffset.get();
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
index d9f32716756..5245ac2853e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
@@ -28,15 +28,13 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.LongAdder;
-
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.regionserver.HeapMemoryManager.HeapMemoryTuneObserver;
import org.apache.hadoop.hbase.util.Bytes;
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;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
/**
@@ -75,15 +73,12 @@ public class ChunkCreator {
private Map chunkIdMap = new ConcurrentHashMap();
private final boolean offheap;
- @VisibleForTesting
static ChunkCreator instance;
- @VisibleForTesting
static boolean chunkPoolDisabled = false;
private MemStoreChunkPool dataChunksPool;
private final int chunkSize;
private MemStoreChunkPool indexChunksPool;
- @VisibleForTesting
ChunkCreator(int chunkSize, boolean offheap, long globalMemStoreSize, float poolSizePercentage,
float initialCountPercentage, HeapMemoryManager heapMemoryManager,
float indexChunkSizePercentage) {
@@ -93,7 +88,6 @@ public class ChunkCreator {
initialCountPercentage, heapMemoryManager);
}
- @VisibleForTesting
private void initializePools(int chunkSize, long globalMemStoreSize,
float poolSizePercentage, float indexChunkSizePercentage,
float initialCountPercentage,
@@ -122,7 +116,6 @@ public class ChunkCreator {
*/
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "LI_LAZY_INIT_STATIC",
justification = "Method is called by single thread at the starting of RS")
- @VisibleForTesting
public static ChunkCreator initialize(int chunkSize, boolean offheap, long globalMemStoreSize,
float poolSizePercentage, float initialCountPercentage,
HeapMemoryManager heapMemoryManager,
@@ -135,7 +128,6 @@ public class ChunkCreator {
return instance;
}
- @VisibleForTesting
public static ChunkCreator getInstance() {
return instance;
}
@@ -280,8 +272,7 @@ public class ChunkCreator {
return createChunk(true, chunkIndexType, chunkSize);
}
- @VisibleForTesting
- // Used to translate the ChunkID into a chunk ref
+ // Used to translate the ChunkID into a chunk ref
Chunk getChunk(int id) {
// can return null if chunk was never mapped
return chunkIdMap.get(id);
@@ -299,14 +290,12 @@ public class ChunkCreator {
return this.chunkIdMap.remove(chunkId);
}
- @VisibleForTesting
- // the chunks in the chunkIdMap may already be released so we shouldn't relay
- // on this counting for strong correctness. This method is used only in testing.
+ // the chunks in the chunkIdMap may already be released so we shouldn't relay
+ // on this counting for strong correctness. This method is used only in testing.
int numberOfMappedChunks() {
return this.chunkIdMap.size();
}
- @VisibleForTesting
void clearChunkIds() {
this.chunkIdMap.clear();
}
@@ -471,7 +460,6 @@ public class ChunkCreator {
}
}
- @VisibleForTesting
static void clearDisableFlag() {
chunkPoolDisabled = false;
}
@@ -507,12 +495,10 @@ public class ChunkCreator {
return memStoreChunkPool;
}
- @VisibleForTesting
int getMaxCount() {
return getMaxCount(ChunkType.DATA_CHUNK);
}
- @VisibleForTesting
int getMaxCount(ChunkType chunkType) {
switch (chunkType) {
case INDEX_CHUNK:
@@ -533,12 +519,10 @@ public class ChunkCreator {
return 0;
}
- @VisibleForTesting
int getPoolSize() {
return getPoolSize(ChunkType.DATA_CHUNK);
}
- @VisibleForTesting
int getPoolSize(ChunkType chunkType) {
switch (chunkType) {
case INDEX_CHUNK:
@@ -558,7 +542,6 @@ public class ChunkCreator {
return 0;
}
- @VisibleForTesting
boolean isChunkInPool(int chunkId) {
Chunk c = getChunk(chunkId);
if (c==null) {
@@ -577,7 +560,6 @@ public class ChunkCreator {
/*
* Only used in testing
*/
- @VisibleForTesting
void clearChunksInPool() {
if (dataChunksPool != null) {
dataChunksPool.reclaimedChunks.clear();
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
index 91519a9eaf6..d40a8828127 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver;
import static org.apache.hadoop.hbase.regionserver.Store.NO_PRIORITY;
import static org.apache.hadoop.hbase.regionserver.Store.PRIORITY_USER;
+
import java.io.IOException;
import java.io.PrintWriter;
import java.io.StringWriter;
@@ -53,7 +54,7 @@ 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;
+
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
@@ -779,12 +780,10 @@ public class CompactSplit implements CompactionRequester, PropagatingConfigurati
// No children to register
}
- @VisibleForTesting
public ThroughputController getCompactionThroughputController() {
return compactionThroughputController;
}
- @VisibleForTesting
/**
* Shutdown the long compaction thread pool.
* Should only be used in unit test to prevent long compaction thread pool from stealing job
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java
index 2f24135ccc7..2be04c22729 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java
@@ -26,7 +26,6 @@ import org.apache.hadoop.hbase.executor.EventType;
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 chore service that periodically cleans up the compacted files when there are no active readers
@@ -38,7 +37,6 @@ public class CompactedHFilesDischarger extends ScheduledChore {
private static final Logger LOG = LoggerFactory.getLogger(CompactedHFilesDischarger.class);
private RegionServerServices regionServerServices;
// Default is to use executor
- @VisibleForTesting
private boolean useExecutor = true;
/**
@@ -59,7 +57,6 @@ public class CompactedHFilesDischarger extends ScheduledChore {
* @param regionServerServices the region server that starts this chore
* @param useExecutor true if to use the region server's executor service, false otherwise
*/
- @VisibleForTesting
public CompactedHFilesDischarger(final int period, final Stoppable stopper,
final RegionServerServices regionServerServices, boolean useExecutor) {
// Need to add the config classes
@@ -73,7 +70,6 @@ public class CompactedHFilesDischarger extends ScheduledChore {
* cleanup. Use this method to set no-executor before you call run.
* @return The old setting for useExecutor
*/
- @VisibleForTesting
boolean setUseExecutor(final boolean useExecutor) {
boolean oldSetting = this.useExecutor;
this.useExecutor = useExecutor;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java
index 6427fa5beac..4ee2db94724 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java
@@ -18,9 +18,6 @@
*/
package org.apache.hadoop.hbase.regionserver;
-import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
@@ -32,13 +29,15 @@ import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MemoryCompactionPolicy;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
/**
* A memstore implementation which supports in-memory compaction.
@@ -79,7 +78,6 @@ public class CompactingMemStore extends AbstractMemStore {
// inWalReplay is true while we are synchronously replaying the edits from WAL
private boolean inWalReplay = false;
- @VisibleForTesting
protected final AtomicBoolean allowCompaction = new AtomicBoolean(true);
private boolean compositeSnapshot = true;
@@ -128,7 +126,6 @@ public class CompactingMemStore extends AbstractMemStore {
(this.compactor == null? "NULL": this.compactor.toString()));
}
- @VisibleForTesting
protected MemStoreCompactor createMemStoreCompactor(MemoryCompactionPolicy compactionPolicy)
throws IllegalArgumentIOException {
return new MemStoreCompactor(this, compactionPolicy);
@@ -334,7 +331,6 @@ public class CompactingMemStore extends AbstractMemStore {
}
// the getSegments() method is used for tests only
- @VisibleForTesting
@Override
protected List getSegments() {
List extends Segment> pipelineList = pipeline.getSegments();
@@ -367,7 +363,6 @@ public class CompactingMemStore extends AbstractMemStore {
}
// setter is used only for testability
- @VisibleForTesting
void setIndexType(IndexType type) {
indexType = type;
// Because this functionality is for testing only and tests are setting in-memory flush size
@@ -413,7 +408,6 @@ public class CompactingMemStore extends AbstractMemStore {
return list;
}
- @VisibleForTesting
protected List createList(int capacity) {
return new ArrayList<>(capacity);
}
@@ -451,7 +445,6 @@ public class CompactingMemStore extends AbstractMemStore {
// externally visible only for tests
// when invoked directly from tests it must be verified that the caller doesn't hold updatesLock,
// otherwise there is a deadlock
- @VisibleForTesting
void flushInMemory() {
MutableSegment currActive = getActive();
if(currActive.setInMemoryFlushed()) {
@@ -499,7 +492,6 @@ public class CompactingMemStore extends AbstractMemStore {
return getRegionServices().getInMemoryCompactionPool();
}
- @VisibleForTesting
protected boolean shouldFlushInMemory(MutableSegment currActive, Cell cellToAdd,
MemStoreSizing memstoreSizing) {
long cellSize = MutableSegment.getCellLength(cellToAdd);
@@ -596,7 +588,6 @@ public class CompactingMemStore extends AbstractMemStore {
}
}
- @VisibleForTesting
boolean isMemStoreFlushingInMemory() {
return inMemoryCompactionInProgress.get();
}
@@ -619,7 +610,6 @@ public class CompactingMemStore extends AbstractMemStore {
return lowest;
}
- @VisibleForTesting
long getInmemoryFlushSize() {
return inmemoryFlushSize;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompositeImmutableSegment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompositeImmutableSegment.java
index 19647faa9de..53ef82d5695 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompositeImmutableSegment.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompositeImmutableSegment.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* The CompositeImmutableSegments is created as a collection of ImmutableSegments and supports
@@ -52,7 +51,6 @@ public class CompositeImmutableSegment extends ImmutableSegment {
}
}
- @VisibleForTesting
@Override
public List getAllSegments() {
return new ArrayList<>(segments);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ConstantSizeRegionSplitPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ConstantSizeRegionSplitPolicy.java
index 5ffd96021fe..1f22dc4d2b2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ConstantSizeRegionSplitPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ConstantSizeRegionSplitPolicy.java
@@ -17,8 +17,6 @@
*/
package org.apache.hadoop.hbase.regionserver;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
import java.util.Random;
import org.apache.hadoop.conf.Configuration;
@@ -86,7 +84,7 @@ public class ConstantSizeRegionSplitPolicy extends RegionSplitPolicy {
return desiredMaxFileSize;
}
- @VisibleForTesting
+ @InterfaceAudience.Private
public boolean positiveJitterRate() {
return this.jitterRate > 0;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 964523554fe..9468130fd76 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -162,6 +162,20 @@ import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
import org.apache.hadoop.hbase.replication.ReplicationUtils;
import org.apache.hadoop.hbase.replication.regionserver.ReplicationObserver;
import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor;
+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.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
import org.apache.hadoop.hbase.trace.TraceUtil;
@@ -186,11 +200,6 @@ import org.apache.hadoop.hbase.wal.WALSplitUtil;
import org.apache.hadoop.hbase.wal.WALSplitUtil.MutationReplay;
import org.apache.hadoop.util.StringUtils;
import org.apache.htrace.core.TraceScope;
-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.collect.Iterables;
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
@@ -200,21 +209,9 @@ import org.apache.hbase.thirdparty.com.google.protobuf.Service;
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.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor.EventType;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
/**
* Regions store data for a certain region of a table. It stores all columns
@@ -315,7 +312,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
// Track data size in all memstores
private final MemStoreSizing memStoreSizing = new ThreadSafeMemStoreSizing();
- @VisibleForTesting
RegionServicesForStores regionServicesForStores;
// Debug possible data loss due to WAL off
@@ -576,7 +572,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
}
/** A result object from prepare flush cache stage */
- @VisibleForTesting
static class PrepareFlushResult {
final FlushResultImpl result; // indicating a failure result from prepare
final TreeMap storeFlushCtxs;
@@ -741,7 +736,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* @deprecated Use other constructors.
*/
@Deprecated
- @VisibleForTesting
public HRegion(final Path tableDir, final WAL wal, final FileSystem fs,
final Configuration confParam, final RegionInfo regionInfo,
final TableDescriptor htd, final RegionServerServices rsServices) {
@@ -946,7 +940,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* @return What the next sequence (edit) id should be.
* @throws IOException e
*/
- @VisibleForTesting
long initialize(final CancelableProgressable reporter) throws IOException {
//Refuse to open the region if there is no column family in the table
@@ -1220,7 +1213,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
return allStoreFiles;
}
- @VisibleForTesting
protected void writeRegionOpenMarker(WAL wal, long openSeqId) throws IOException {
Map> storeFiles = getStoreFiles();
RegionEventDescriptor regionOpenDesc = ProtobufUtil.toRegionEventDescriptor(
@@ -1499,7 +1491,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
}
}
- @VisibleForTesting
public MultiVersionConcurrencyControl getMVCC() {
return mvcc;
}
@@ -1616,7 +1607,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
/**
* Exposed for some very specific unit tests.
*/
- @VisibleForTesting
public void setClosing(boolean closing) {
this.closing.set(closing);
}
@@ -1626,7 +1616,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* Instead of blocking, the {@link HRegion#doClose} will throw exception if you set the timeout.
* @param timeoutForWriteLock the second time to wait for the write lock in {@link HRegion#doClose}
*/
- @VisibleForTesting
public void setTimeoutForWriteLock(long timeoutForWriteLock) {
assert timeoutForWriteLock >= 0;
this.timeoutForWriteLock = timeoutForWriteLock;
@@ -2024,7 +2013,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
return this.htableDescriptor;
}
- @VisibleForTesting
void setTableDescriptor(TableDescriptor desc) {
htableDescriptor = desc;
}
@@ -2041,7 +2029,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
/**
* Only used for unit test which doesn't start region server.
*/
- @VisibleForTesting
public void setBlockCache(BlockCache blockCache) {
this.blockCache = blockCache;
}
@@ -2053,7 +2040,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
/**
* Only used for unit test which doesn't start region server.
*/
- @VisibleForTesting
public void setMobFileCache(MobFileCache mobFileCache) {
this.mobFileCache = mobFileCache;
}
@@ -2061,7 +2047,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
/**
* @return split policy for this region.
*/
- @VisibleForTesting
RegionSplitPolicy getSplitPolicy() {
return this.splitPolicy;
}
@@ -2105,7 +2090,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* @return the Region directory under WALRootDirectory
* @throws IOException if there is an error getting WALRootDir
*/
- @VisibleForTesting
public Path getWALRegionDir() throws IOException {
if (regionDir == null) {
regionDir = CommonFSUtils.getWALRegionDir(conf, getRegionInfo().getTable(),
@@ -2214,7 +2198,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
*
* It is used by utilities and testing
*/
- @VisibleForTesting
public void compactStores() throws IOException {
for (HStore s : stores.values()) {
Optional compaction = s.requestCompaction();
@@ -2229,7 +2212,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
*
* It is used by utilities and testing
*/
- @VisibleForTesting
void compactStore(byte[] family, ThroughputController throughputController) throws IOException {
HStore s = getStore(family);
Optional compaction = s.requestCompaction();
@@ -3095,7 +3077,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* @return Next sequence number unassociated with any actual edit.
* @throws IOException
*/
- @VisibleForTesting
protected long getNextSequenceId(final WAL wal) throws IOException {
WriteEntry we = mvcc.begin();
mvcc.completeAndWait(we);
@@ -5257,7 +5238,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* @return the sequence id of the last edit added to this region out of the
* recovered edits log or minSeqId if nothing added from editlogs.
*/
- @VisibleForTesting
long replayRecoveredEditsIfAny(Map maxSeqIdInStores,
final CancelableProgressable reporter, final MonitoredTask status) throws IOException {
long minSeqIdForTheRegion = -1;
@@ -5723,7 +5703,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* the store memstores, only if the memstores do not have a higher seqId from an earlier wal
* edit (because the events may be coming out of order).
*/
- @VisibleForTesting
PrepareFlushResult replayWALFlushStartMarker(FlushDescriptor flush) throws IOException {
long flushSeqId = flush.getFlushSequenceNumber();
@@ -5834,7 +5813,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
return null;
}
- @VisibleForTesting
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
justification="Intentional; post memstore flush")
void replayWALFlushCommitMarker(FlushDescriptor flush) throws IOException {
@@ -6122,7 +6100,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
}
}
- @VisibleForTesting
PrepareFlushResult getPrepareFlushResult() {
return prepareFlushResult;
}
@@ -6486,7 +6463,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* @param s Store to add edit too.
* @param cell Cell to add.
*/
- @VisibleForTesting
protected void restoreEdit(HStore s, Cell cell, MemStoreSizing memstoreAccounting) {
s.add(cell, memstoreAccounting);
}
@@ -6689,7 +6665,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
}
}
- @VisibleForTesting
public int getReadLockCount() {
return lock.getReadLockCount();
}
@@ -6698,7 +6673,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
return lockedRows;
}
- @VisibleForTesting
class RowLockContext {
private final HashedBytes row;
final ReadWriteLock readWriteLock = new ReentrantReadWriteLock(true);
@@ -6775,7 +6749,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
return lock;
}
- @VisibleForTesting
public RowLockContext getContext() {
return context;
}
@@ -8064,7 +8037,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
return r.openHRegion(reporter);
}
- @VisibleForTesting
public NavigableMap getReplicationScope() {
return this.replicationScope;
}
@@ -8861,7 +8833,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
}
/** @param coprocessorHost the new coprocessor host */
- @VisibleForTesting
public void setCoprocessorHost(final RegionCoprocessorHost coprocessorHost) {
this.coprocessorHost = coprocessorHost;
}
@@ -9161,7 +9132,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
flushesQueued.increment();
}
- @VisibleForTesting
public long getReadPoint() {
return getReadPoint(IsolationLevel.READ_COMMITTED);
}
@@ -9361,12 +9331,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
}
}
- @VisibleForTesting
public void setReadRequestsCount(long readRequestsCount) {
this.readRequestsCount.add(readRequestsCount);
}
- @VisibleForTesting
public void setWriteRequestsCount(long writeRequestsCount) {
this.writeRequestsCount.add(writeRequestsCount);
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
index d5ef30ecc79..73234f161c4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
@@ -54,7 +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;
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
/**
@@ -75,7 +74,7 @@ public class HRegionFileSystem {
public static final String REGION_SPLITS_DIR = ".splits";
/** Temporary subdirectory of the region directory used for compaction output. */
- @VisibleForTesting static final String REGION_TEMP_DIR = ".tmp";
+ static final String REGION_TEMP_DIR = ".tmp";
private final RegionInfo regionInfo;
//regionInfo for interacting with FS (getting encodedName, etc)
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index be98f77a46d..fae72d1f647 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -174,7 +174,6 @@ import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.hadoop.hbase.wal.NettyAsyncFSWALConfigHelper;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALFactory;
-import org.apache.hadoop.hbase.wal.WALProvider;
import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
@@ -190,7 +189,7 @@ import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import sun.misc.Signal;
-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.Throwables;
import org.apache.hbase.thirdparty.com.google.common.cache.Cache;
@@ -201,6 +200,7 @@ import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
+
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall;
@@ -246,7 +246,7 @@ public class HRegionServer extends Thread implements
/**
* For testing only! Set to true to skip notifying region assignment to master .
*/
- @VisibleForTesting
+ @InterfaceAudience.Private
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="MS_SHOULD_BE_FINAL")
public static boolean TEST_SKIP_REPORTING_TRANSITION = false;
@@ -542,7 +542,7 @@ public class HRegionServer extends Thread implements
*/
protected final ConfigurationManager configurationManager;
- @VisibleForTesting
+ @InterfaceAudience.Private
CompactedHFilesDischarger compactedFileDischarger;
private volatile ThroughputController flushThroughputController;
@@ -1254,7 +1254,7 @@ public class HRegionServer extends Thread implements
return writeCount;
}
- @VisibleForTesting
+ @InterfaceAudience.Private
protected void tryRegionServerReport(long reportStartTime, long reportEndTime)
throws IOException {
RegionServerStatusService.BlockingInterface rss = rssStub;
@@ -2509,7 +2509,7 @@ public class HRegionServer extends Thread implements
return rpcServices.rpcServer;
}
- @VisibleForTesting
+ @InterfaceAudience.Private
public RSRpcServices getRSRpcServices() {
return rpcServices;
}
@@ -2599,7 +2599,7 @@ public class HRegionServer extends Thread implements
* logs but it does close socket in case want to bring up server on old
* hostname+port immediately.
*/
- @VisibleForTesting
+ @InterfaceAudience.Private
protected void kill() {
this.killed = true;
abort("Simulated kill");
@@ -2674,7 +2674,7 @@ public class HRegionServer extends Thread implements
* @return Return the object that implements the replication
* source executorService.
*/
- @VisibleForTesting
+ @InterfaceAudience.Private
public ReplicationSourceService getReplicationSourceService() {
return replicationSourceHandler;
}
@@ -2707,7 +2707,7 @@ public class HRegionServer extends Thread implements
* @param refresh If true then master address will be read from ZK, otherwise use cached data
* @return master + port, or null if server has been stopped
*/
- @VisibleForTesting
+ @InterfaceAudience.Private
protected synchronized ServerName createRegionServerStatusStub(boolean refresh) {
if (rssStub != null) {
return masterAddressTracker.getMasterAddress();
@@ -3540,12 +3540,12 @@ public class HRegionServer extends Thread implements
movedRegionInfoCache.invalidate(encodedName);
}
- @VisibleForTesting
+ @InterfaceAudience.Private
public MovedRegionInfo getMovedRegion(String encodedRegionName) {
return movedRegionInfoCache.getIfPresent(encodedRegionName);
}
- @VisibleForTesting
+ @InterfaceAudience.Private
public int movedRegionCacheExpiredTime() {
return TIMEOUT_REGION_MOVED;
}
@@ -3641,7 +3641,7 @@ public class HRegionServer extends Thread implements
/**
* @return : Returns the ConfigurationManager object for testing purposes.
*/
- @VisibleForTesting
+ @InterfaceAudience.Private
ConfigurationManager getConfigurationManager() {
return configurationManager;
}
@@ -3705,7 +3705,7 @@ public class HRegionServer extends Thread implements
* For testing
* @return whether all wal roll request finished for this regionserver
*/
- @VisibleForTesting
+ @InterfaceAudience.Private
public boolean walRollRequestFinished() {
return this.walRoller.walRollFinished();
}
@@ -3905,7 +3905,7 @@ public class HRegionServer extends Thread implements
}
}
- @VisibleForTesting
+ @InterfaceAudience.Private
public CompactedHFilesDischarger getCompactedHFilesDischarger() {
return compactedFileDischarger;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index 55d6865d5f3..5b685df9864 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -108,7 +108,7 @@ import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
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.collect.ImmutableCollection;
import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
@@ -117,8 +117,10 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
import org.apache.hbase.thirdparty.org.apache.commons.collections4.IterableUtils;
+
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
+
/**
* A Store holds a column family in a Region. Its a memstore and a set of zero
* or more StoreFiles, which stretch backwards over time.
@@ -739,7 +741,6 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
completeCompaction(toBeRemovedStoreFiles);
}
- @VisibleForTesting
protected HStoreFile createStoreFileAndReader(final Path p) throws IOException {
StoreFileInfo info = new StoreFileInfo(conf, this.getFileSystem(),
p, isPrimaryReplicaStore());
@@ -1552,7 +1553,6 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
}
}
- @VisibleForTesting
protected List doCompaction(CompactionRequestImpl cr,
Collection filesToCompact, User user, long compactionStartTime,
List newFiles) throws IOException {
@@ -1647,7 +1647,6 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
this.region.getRegionInfo(), compactionDescriptor, this.region.getMVCC());
}
- @VisibleForTesting
void replaceStoreFiles(Collection compactedFiles, Collection result)
throws IOException {
this.lock.writeLock().lock();
@@ -1808,7 +1807,6 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
* but instead makes a compaction candidate list by itself.
* @param N Number of files.
*/
- @VisibleForTesting
public void compactRecentForTestingAssumingDefaultPolicy(int N) throws IOException {
List filesToCompact;
boolean isMajor;
@@ -2083,7 +2081,6 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
* Update counts.
* @param compactedFiles list of files that were compacted
*/
- @VisibleForTesting
protected void completeCompaction(Collection compactedFiles)
// Rename this method! TODO.
throws IOException {
@@ -2575,7 +2572,6 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
* Used for tests.
* @return cache configuration for this Store.
*/
- @VisibleForTesting
public CacheConfig getCacheConfig() {
return this.cacheConf;
}
@@ -2655,7 +2651,6 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
* Returns the StoreEngine that is backing this concrete implementation of Store.
* @return Returns the {@link StoreEngine} object used internally inside this HStore object.
*/
- @VisibleForTesting
public StoreEngine, ?, ?, ?> getStoreEngine() {
return this.storeEngine;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java
index cf73f0f6887..9f8a717d21f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java
@@ -41,12 +41,15 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.ReaderContext;
import org.apache.hadoop.hbase.io.hfile.ReaderContext.ReaderType;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.util.BloomFilterFactory;
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.hbase.thirdparty.com.google.common.base.Preconditions;
+
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
/**
@@ -319,7 +322,6 @@ public class HStoreFile implements StoreFile {
return compactedAway;
}
- @VisibleForTesting
public int getRefCount() {
return fileInfo.refCount.get();
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java
index ea5586c4af9..1f831eefee6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java
@@ -39,8 +39,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;
-
/**
* Manages tuning of Heap memory using HeapMemoryTuner . Most part of the heap memory is
* split between Memstores and BlockCache. This manager helps in tuning sizes of both these
@@ -106,7 +104,6 @@ public class HeapMemoryManager {
private List tuneObservers = new ArrayList<>();
- @VisibleForTesting
HeapMemoryManager(BlockCache blockCache, FlushRequester memStoreFlusher,
Server server, RegionServerAccounting regionServerAccounting) {
Configuration conf = server.getConfiguration();
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java
index f28e28e31cc..2fef5d106f0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java
@@ -19,8 +19,6 @@
package org.apache.hadoop.hbase.regionserver;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
import java.io.IOException;
import java.util.ArrayList;
import java.util.Comparator;
@@ -29,10 +27,10 @@ import java.util.PriorityQueue;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState;
/**
* Implements a heap merge across any number of KeyValueScanners.
@@ -419,8 +417,6 @@ public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner
return this.heap;
}
-
- @VisibleForTesting
KeyValueScanner getCurrentForTesting() {
return current;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java
index cbab595517f..be73bbaf8a3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java
@@ -27,8 +27,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;
-
/**
* Runs periodically to determine if the WAL should be rolled.
*
@@ -39,7 +37,6 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
* TODO: change to a pool of threads
*/
@InterfaceAudience.Private
-@VisibleForTesting
public class LogRoller extends AbstractWALRoller {
private static final Logger LOG = LoggerFactory.getLogger(LogRoller.class);
@@ -64,7 +61,6 @@ public class LogRoller extends AbstractWALRoller {
requester.requestFlush(r, families, FlushLifeCycleTracker.DUMMY);
}
- @VisibleForTesting
Map getWalNeedsRoll() {
return this.wals;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactor.java
index 45d432c4369..4df9ae0b60c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactor.java
@@ -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;
-
/**
* The ongoing MemStore Compaction manager, dispatches a solo running compaction and interrupts
* the compaction if requested. The compaction is interrupted and stopped by CompactingMemStore,
@@ -242,7 +240,6 @@ public class MemStoreCompactor {
return result;
}
- @VisibleForTesting
void initiateCompactionStrategy(MemoryCompactionPolicy compType,
Configuration configuration, String cfName) throws IllegalArgumentIOException {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
index 7951c72b0da..f5fccf4d2a8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
@@ -27,7 +27,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.ReentrantLock;
-
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ByteBufferExtendedCell;
import org.apache.hadoop.hbase.Cell;
@@ -36,8 +35,9 @@ import org.apache.hadoop.hbase.KeyValueUtil;
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;
+
/**
* A memstore-local allocation buffer.
*
@@ -73,7 +73,6 @@ public class MemStoreLABImpl implements MemStoreLAB {
private ReentrantLock lock = new ReentrantLock();
// A set of chunks contained by this memstore LAB
- @VisibleForTesting
Set chunks = new ConcurrentSkipListSet();
private final int dataChunkSize;
private final int maxAlloc;
@@ -270,7 +269,6 @@ public class MemStoreLABImpl implements MemStoreLAB {
}
}
- @VisibleForTesting
int getOpenScannerCount() {
return this.openScannerCount.get();
}
@@ -397,12 +395,10 @@ public class MemStoreLABImpl implements MemStoreLAB {
return this.chunkCreator.isOffheap();
}
- @VisibleForTesting
Chunk getCurrentChunk() {
return currChunk.get();
}
- @VisibleForTesting
BlockingQueue getPooledChunks() {
BlockingQueue pooledChunks = new LinkedBlockingQueue<>();
for (Integer id : this.chunks) {
@@ -414,7 +410,7 @@ public class MemStoreLABImpl implements MemStoreLAB {
return pooledChunks;
}
- @VisibleForTesting Integer getNumOfChunksReturnedToPool() {
+ Integer getNumOfChunksReturnedToPool() {
int i = 0;
for (Integer id : this.chunks) {
if (chunkCreator.isChunkInPool(id)) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java
index 172a047a546..5b05c12348b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java
@@ -27,8 +27,6 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
/**
*
* This class is for maintaining the various regionserver statistics
@@ -95,12 +93,10 @@ public class MetricsRegionServer {
return null;
}
- @VisibleForTesting
public MetricsRegionServerSource getMetricsSource() {
return serverSource;
}
- @VisibleForTesting
public MetricsUserAggregate getMetricsUserAggregate() {
return userAggregate;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiVersionConcurrencyControl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiVersionConcurrencyControl.java
index 2b350e686f9..d821eecf6f3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiVersionConcurrencyControl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiVersionConcurrencyControl.java
@@ -18,19 +18,16 @@
*/
package org.apache.hadoop.hbase.regionserver;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects;
-import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects.ToStringHelper;
-
import java.util.LinkedList;
import java.util.concurrent.atomic.AtomicLong;
-
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.ClassSize;
+import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects;
+import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects.ToStringHelper;
/**
* Manages the read/write consistency. This provides an interface for readers to determine what
@@ -251,7 +248,6 @@ public class MultiVersionConcurrencyControl {
}
}
- @VisibleForTesting
@Override
public String toString() {
ToStringHelper helper = MoreObjects.toStringHelper(this).add("readPoint", readPoint)
@@ -266,7 +262,6 @@ public class MultiVersionConcurrencyControl {
return readPoint.get();
}
- @VisibleForTesting
public long getWritePoint() {
return writePoint.get();
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableSegment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableSegment.java
index 2c1e859fbd4..6e813433fac 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableSegment.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableSegment.java
@@ -28,10 +28,8 @@ import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.KeyValue;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.ClassSize;
-
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* A mutable segment in memstore, specifically the active segment.
@@ -124,7 +122,6 @@ public class MutableSegment extends Segment {
* Returns the first cell in the segment
* @return the first cell in the segment
*/
- @VisibleForTesting
Cell first() {
return this.getCellSet().first();
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index ac79afab9d2..c9a710adc35 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -152,7 +152,6 @@ 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.cache.Cache;
import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder;
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
@@ -324,7 +323,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
final RpcServerInterface rpcServer;
final InetSocketAddress isa;
- @VisibleForTesting
protected final HRegionServer regionServer;
private final long maxScannerResultSize;
@@ -1232,7 +1230,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
}
}
- @VisibleForTesting
public int getScannersCount() {
return scanners.size();
}
@@ -1370,7 +1367,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
* @throws IOException if the specifier is not null,
* but failed to find the region
*/
- @VisibleForTesting
public HRegion getRegion(
final RegionSpecifier regionSpecifier) throws IOException {
return regionServer.getRegion(regionSpecifier.getValue().toByteArray());
@@ -1397,12 +1393,10 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
return regions;
}
- @VisibleForTesting
public PriorityFunction getPriority() {
return priority;
}
- @VisibleForTesting
public Configuration getConfiguration() {
return regionServer.getConfiguration();
}
@@ -3856,7 +3850,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
throw new ServiceException("Invalid request params");
}
- @VisibleForTesting
public RpcScheduler getRpcScheduler() {
return rpcServer.getScheduler();
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java
index 36392d7ef73..06795a58545 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.io.ByteBuffAllocator;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
/**
@@ -113,7 +112,6 @@ public class RegionServicesForStores {
return region.getTableDescriptor().getColumnFamilyCount();
}
- @VisibleForTesting
long getMemStoreSize() {
return region.getMemStoreDataSize();
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanInfo.java
index 7d6161849ad..4e2066601c5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanInfo.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanInfo.java
@@ -18,7 +18,6 @@
package org.apache.hadoop.hbase.regionserver;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
@@ -26,15 +25,13 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeepDeletedCells;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
-
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.yetus.audience.InterfaceAudience;
/**
* Immutable information for scans over a store.
*/
// Has to be public for PartitionedMobCompactor to access; ditto on tests making use of a few of
// the accessors below. Shutdown access. TODO
-@VisibleForTesting
@InterfaceAudience.Private
public class ScanInfo {
private byte[] family;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java
index 40ba3888579..f74e2f89be3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java
@@ -57,7 +57,7 @@ import org.apache.hadoop.security.token.Token;
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.ClientProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest;
@@ -165,12 +165,10 @@ public class SecureBulkLoadManager {
private Consumer fsCreatedListener;
- @VisibleForTesting
void setFsCreatedListener(Consumer fsCreatedListener) {
this.fsCreatedListener = fsCreatedListener;
}
-
private void incrementUgiReference(UserGroupInformation ugi) {
// if we haven't seen this ugi before, make a new counter
ugiReferenceCounter.compute(ugi, (key, value) -> {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java
index b1c92a4e8d5..b0763aa3835 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java
@@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* This is an abstraction of a segment maintained in a memstore, e.g., the active
@@ -181,7 +180,6 @@ public abstract class Segment implements MemStoreSizing {
/**
* Get cell length after serialized in {@link KeyValue}
*/
- @VisibleForTesting
static int getCellLength(Cell cell) {
return cell.getSerializedSize();
}
@@ -414,7 +412,6 @@ public abstract class Segment implements MemStoreSizing {
return getCellSet().tailSet(firstCell);
}
- @VisibleForTesting
MemStoreLAB getMemStoreLAB() {
return memStoreLAB;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ServerNonceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ServerNonceManager.java
index 0b1d251d989..e0ea974b194 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ServerNonceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ServerNonceManager.java
@@ -27,13 +27,11 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ScheduledChore;
import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.NonceKey;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.NonceKey;
-
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Implementation of nonce manager that stores nonces in a hash map and cleans them up after
@@ -134,7 +132,6 @@ public class ServerNonceManager {
}
}
- @VisibleForTesting
public void setConflictWaitIterationMs(int conflictWaitIterationMs) {
this.conflictWaitIterationMs = conflictWaitIterationMs;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java
index ba2c2b49e26..b28a4c4fbc4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java
@@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.wal.WALSplitter;
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 worker is spawned in every regionserver, including master. The Worker waits for log
@@ -211,7 +210,6 @@ public class SplitLogWorker implements Runnable {
* Returns the number of tasks processed by coordination.
* This method is used by tests only
*/
- @VisibleForTesting
public int getTaskReadySeq() {
return coordination.getTaskReadySeq();
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java
index f92a4d38659..7550511a356 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java
@@ -54,7 +54,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;
/**
* Reader for a StoreFile.
@@ -115,7 +114,7 @@ public class StoreFileReader {
/**
* ONLY USE DEFAULT CONSTRUCTOR FOR UNIT TESTS
*/
- @VisibleForTesting
+ @InterfaceAudience.Private
StoreFileReader() {
this.refCount = new AtomicInteger(0);
this.reader = null;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
index 2935949379d..8ea4cd202ec 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
@@ -25,7 +25,6 @@ import java.util.List;
import java.util.NavigableSet;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.locks.ReentrantLock;
-
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil;
@@ -46,11 +45,9 @@ import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher;
import org.apache.hadoop.hbase.regionserver.querymatcher.UserScanQueryMatcher;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
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.org.apache.commons.collections4.CollectionUtils;
@@ -93,9 +90,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
private final int minVersions;
private final long maxRowSize;
private final long cellsPerHeartbeatCheck;
- @VisibleForTesting
long memstoreOnlyReads;
- @VisibleForTesting
long mixedReads;
// 1) Collects all the KVHeap that are eagerly getting closed during the
@@ -155,7 +150,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
// Since CompactingMemstore is now default, we get three memstore scanners from a flush
private final List memStoreScannersAfterFlush = new ArrayList<>(3);
// The current list of scanners
- @VisibleForTesting
final List currentScanners = new ArrayList<>();
// flush update lock
private final ReentrantLock flushLock = new ReentrantLock();
@@ -344,7 +338,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
}
// Used to instantiate a scanner for user scan in test
- @VisibleForTesting
StoreScanner(Scan scan, ScanInfo scanInfo, NavigableSet columns,
List extends KeyValueScanner> scanners) throws IOException {
// 0 is passed as readpoint because the test bypasses Store
@@ -356,7 +349,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
}
// Used to instantiate a scanner for user scan in test
- @VisibleForTesting
StoreScanner(Scan scan, ScanInfo scanInfo, NavigableSet columns,
List extends KeyValueScanner> scanners, ScanType scanType) throws IOException {
// 0 is passed as readpoint because the test bypasses Store
@@ -373,7 +365,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
}
// Used to instantiate a scanner for compaction in test
- @VisibleForTesting
StoreScanner(ScanInfo scanInfo, int maxVersions, ScanType scanType,
List extends KeyValueScanner> scanners) throws IOException {
// 0 is passed as readpoint because the test bypasses Store
@@ -384,10 +375,10 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
seekAllScanner(scanInfo, scanners);
}
- @VisibleForTesting
boolean isScanUsePread() {
return this.scanUsePread;
}
+
/**
* Seek the specified scanners with the given key
* @param scanners
@@ -427,7 +418,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
}
}
- @VisibleForTesting
protected void resetKVHeap(List extends KeyValueScanner> scanners,
CellComparator comparator) throws IOException {
// Combine all seeked scanners with a heap
@@ -444,7 +434,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
*
* Will be overridden by testcase so declared as protected.
*/
- @VisibleForTesting
protected List selectScannersFrom(HStore store,
List extends KeyValueScanner> allScanners) {
boolean memOnly;
@@ -869,7 +858,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
* @param cell current cell
* @return true means skip to next row, false means not
*/
- @VisibleForTesting
protected boolean trySkipToNextRow(Cell cell) throws IOException {
Cell nextCell = null;
// used to guard against a changed next indexed key by doing a identity comparison
@@ -895,7 +883,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
* @param cell current cell
* @return true means skip to next column, false means not
*/
- @VisibleForTesting
protected boolean trySkipToNextColumn(Cell cell) throws IOException {
Cell nextCell = null;
// used to guard against a changed next indexed key by doing a identity comparison
@@ -1095,7 +1082,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
return heap.reseek(kv);
}
- @VisibleForTesting
void trySwitchToStreamRead() {
if (readType != Scan.ReadType.DEFAULT || !scanUsePread || closing ||
heap.peek() == null || bytesRead < preadMaxBytes) {
@@ -1207,7 +1193,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
* Used in testing.
* @return all scanners in no particular order
*/
- @VisibleForTesting
List getAllScannersForTesting() {
List allScanners = new ArrayList<>();
KeyValueScanner current = heap.getCurrentForTesting();
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java
index 7beec5e2e8a..1560aef5f6b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java
@@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Stripe implementation of StoreFlusher. Flushes files either into L0 file w/o metadata, or
@@ -119,7 +118,6 @@ public class StripeStoreFlusher extends StoreFlusher {
this.comparator = comparator;
}
- @VisibleForTesting
public StripeMultiFileWriter createWriter() throws IOException {
StripeMultiFileWriter writer = new StripeMultiFileWriter.SizeMultiWriter(comparator, 1,
Long.MAX_VALUE, OPEN_KEY, OPEN_KEY);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/TimeRangeTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/TimeRangeTracker.java
index 18175648f30..6a38be1f3b3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/TimeRangeTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/TimeRangeTracker.java
@@ -24,14 +24,13 @@ import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicLong;
-
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.io.TimeRange;
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.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
@@ -243,7 +242,6 @@ public abstract class TimeRangeTracker {
return new TimeRange(min, max);
}
- @VisibleForTesting
//In order to estimate the heap size, this inner class need to be accessible to TestHeapSize.
public static class NonSyncTimeRangeTracker extends TimeRangeTracker {
private long minimumTimestamp = INITIAL_MIN_TIMESTAMP;
@@ -301,7 +299,6 @@ public abstract class TimeRangeTracker {
}
}
- @VisibleForTesting
//In order to estimate the heap size, this inner class need to be accessible to TestHeapSize.
public static class SyncTimeRangeTracker extends TimeRangeTracker {
private final AtomicLong minimumTimestamp = new AtomicLong(INITIAL_MIN_TIMESTAMP);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/VersionedSegmentsList.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/VersionedSegmentsList.java
index 2a684da0131..d5be356f93f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/VersionedSegmentsList.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/VersionedSegmentsList.java
@@ -21,8 +21,6 @@ package org.apache.hadoop.hbase.regionserver;
import java.util.List;
import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
/**
* A list of segment managers coupled with the version of the memstore (version at the time it was
* created).
@@ -65,7 +63,6 @@ public class VersionedSegmentsList {
}
// Estimates fraction of unique keys
- @VisibleForTesting
double getEstimatedUniquesFrac() {
int segmentCells = 0;
int maxCells = 0;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CurrentHourProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CurrentHourProvider.java
index b68363498e4..1f5a8208f0a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CurrentHourProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CurrentHourProvider.java
@@ -22,7 +22,6 @@ import java.util.GregorianCalendar;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
@InterfaceAudience.Private
public class CurrentHourProvider {
@@ -38,7 +37,6 @@ public class CurrentHourProvider {
}
}
- @VisibleForTesting
static Tick nextTick() {
Calendar calendar = new GregorianCalendar();
calendar.setTimeInMillis(EnvironmentEdgeManager.currentTime());
@@ -54,7 +52,6 @@ public class CurrentHourProvider {
calendar.set(Calendar.MILLISECOND, 0);
}
- @VisibleForTesting
static volatile Tick tick = nextTick();
public static int getCurrentHour() {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionPolicy.java
index 1cc7dda0948..2c6046befd7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionPolicy.java
@@ -26,7 +26,6 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.OptionalLong;
-
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
@@ -40,7 +39,7 @@ import org.apache.hadoop.hbase.util.ReflectionUtils;
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.Iterators;
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
import org.apache.hbase.thirdparty.com.google.common.collect.PeekingIterator;
@@ -97,8 +96,8 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy {
/**
* Heuristics for guessing whether we need minor compaction.
*/
+ @InterfaceAudience.Private
@Override
- @VisibleForTesting
public boolean needsCompaction(Collection storeFiles,
List filesCompacting) {
ArrayList candidates = new ArrayList<>(storeFiles);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/throttle/StoreHotnessProtector.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/throttle/StoreHotnessProtector.java
index 71fd89b5c95..f75fb3c848f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/throttle/StoreHotnessProtector.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/throttle/StoreHotnessProtector.java
@@ -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;
-
/**
* StoreHotnessProtector is designed to help limit the concurrency of puts with dense columns, it
* does best-effort to avoid exhausting all RS's handlers. When a lot of clients write requests with
@@ -189,7 +187,6 @@ public class StoreHotnessProtector {
return this.parallelPutToStoreThreadLimit > 0;
}
- @VisibleForTesting
Map getPreparePutToStoreMap() {
return preparePutToStoreMap;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
index ee8790f5852..6c7f0083d86 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
@@ -48,7 +48,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.ReentrantLock;
-
import org.apache.commons.lang3.mutable.MutableLong;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
@@ -88,7 +87,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.util.concurrent.ThreadFactoryBuilder;
/**
@@ -588,7 +586,6 @@ public abstract class AbstractFSWAL implements WAL {
return newPath;
}
- @VisibleForTesting
Path getOldPath() {
long currentFilenum = this.filenum.get();
Path oldPath = null;
@@ -738,7 +735,6 @@ public abstract class AbstractFSWAL implements WAL {
return new Path(archiveDir, p.getName());
}
- @VisibleForTesting
protected void archiveLogFile(final Path p) throws IOException {
Path newPath = getWALArchivePath(this.walArchiveDir, p);
// Tell our listeners that a log is going to be archived.
@@ -790,7 +786,6 @@ public abstract class AbstractFSWAL implements WAL {
* @return the passed in newPath
* @throws IOException if there is a problem flushing or closing the underlying FS
*/
- @VisibleForTesting
Path replaceWriter(Path oldPath, Path newPath, W nextWriter) throws IOException {
try (TraceScope scope = TraceUtil.createTrace("FSHFile.replaceWriter")) {
doReplaceWriter(oldPath, newPath, nextWriter);
@@ -895,7 +890,6 @@ public abstract class AbstractFSWAL implements WAL {
* Get the backing files associated with this WAL.
* @return may be null if there are no files.
*/
- @VisibleForTesting
FileStatus[] getFiles() throws IOException {
return CommonFSUtils.listStatus(fs, walDir, ourFiles);
}
@@ -994,7 +988,6 @@ public abstract class AbstractFSWAL implements WAL {
/**
* Exposed for testing only. Use to tricks like halt the ring buffer appending.
*/
- @VisibleForTesting
protected void atHeadOfRingBufferEventHandlerAppend() {
// Noop
}
@@ -1249,13 +1242,11 @@ public abstract class AbstractFSWAL implements WAL {
/**
* This method gets the pipeline for the current WAL.
*/
- @VisibleForTesting
abstract DatanodeInfo[] getPipeline();
/**
* This method gets the datanode replication count for the current WAL.
*/
- @VisibleForTesting
abstract int getLogReplication();
private static void split(final Configuration conf, final Path p) throws IOException {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
index f2ee0aee888..7e120979164 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
@@ -40,7 +40,6 @@ import java.util.concurrent.Executors;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
-
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
@@ -61,12 +60,12 @@ import org.apache.hadoop.hdfs.DFSOutputStream;
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.htrace.core.TraceScope;
+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.util.concurrent.ThreadFactoryBuilder;
/**
* The default implementation of FSWAL.
@@ -204,13 +203,11 @@ public class FSHLog extends AbstractFSWAL {
* @param logDir dir where wals are stored
* @param conf configuration to use
*/
- @VisibleForTesting
public FSHLog(final FileSystem fs, final Path root, final String logDir, final Configuration conf)
throws IOException {
this(fs, root, logDir, HConstants.HREGION_OLDLOGDIR_NAME, conf, null, true, null, null);
}
- @VisibleForTesting
public FSHLog(final FileSystem fs, Abortable abortable, final Path root, final String logDir,
final Configuration conf) throws IOException {
this(fs, abortable, root, logDir, HConstants.HREGION_OLDLOGDIR_NAME, conf, null, true, null,
@@ -282,7 +279,6 @@ public class FSHLog extends AbstractFSWAL {
* removed.
* @return null if underlying stream is not ready.
*/
- @VisibleForTesting
OutputStream getOutputStream() {
FSDataOutputStream fsdos = this.hdfs_out;
return fsdos != null ? fsdos.getWrappedStream() : null;
@@ -320,14 +316,12 @@ public class FSHLog extends AbstractFSWAL {
* Used to manufacture race condition reliably. For testing only.
* @see #beforeWaitOnSafePoint()
*/
- @VisibleForTesting
protected void afterCreatingZigZagLatch() {
}
/**
* @see #afterCreatingZigZagLatch()
*/
- @VisibleForTesting
protected void beforeWaitOnSafePoint() {
};
@@ -772,7 +766,6 @@ public class FSHLog extends AbstractFSWAL {
return logRollNeeded;
}
- @VisibleForTesting
protected long getSequenceOnRingBuffer() {
return this.disruptor.getRingBuffer().next();
}
@@ -782,7 +775,6 @@ public class FSHLog extends AbstractFSWAL {
return publishSyncOnRingBuffer(sequence, forceSync);
}
- @VisibleForTesting
protected SyncFuture publishSyncOnRingBuffer(long sequence, boolean forceSync) {
// here we use ring buffer sequence as transaction id
SyncFuture syncFuture = getSyncFuture(sequence, forceSync);
@@ -809,7 +801,6 @@ public class FSHLog extends AbstractFSWAL {
* patch.
*/
@Override
- @VisibleForTesting
int getLogReplication() {
try {
// in standalone mode, it will return 0
@@ -850,7 +841,6 @@ public class FSHLog extends AbstractFSWAL {
}
}
- @VisibleForTesting
boolean isLowReplicationRollEnabled() {
return lowReplicationRollEnabled;
}
@@ -1206,12 +1196,10 @@ public class FSHLog extends AbstractFSWAL {
return new DatanodeInfo[0];
}
- @VisibleForTesting
Writer getWriter() {
return this.writer;
}
- @VisibleForTesting
void setWriter(Writer writer) {
this.writer = writer;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
index 44c96dee761..ca51ec0c568 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
@@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.hadoop.hbase.wal.WALKeyImpl;
import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
/**
@@ -91,7 +90,6 @@ class FSWALEntry extends Entry {
}
}
- @VisibleForTesting
static Set collectFamilies(List cells) {
if (CollectionUtils.isEmpty(cells)) {
return Collections.emptySet();
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWAL.java
index ee99aa30f98..ce82a6c2f55 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWAL.java
@@ -19,8 +19,6 @@
package org.apache.hadoop.hbase.regionserver.wal;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
import java.io.IOException;
import org.apache.hadoop.hbase.TableName;
@@ -46,7 +44,6 @@ public class MetricsWAL implements WALActionsListener {
this(CompatibilitySingletonFactory.getInstance(MetricsWALSource.class));
}
- @VisibleForTesting
MetricsWAL(MetricsWALSource s) {
this.source = s;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceIdAccounting.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceIdAccounting.java
index 27a4c8d442d..d96aafe5b76 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceIdAccounting.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceIdAccounting.java
@@ -36,8 +36,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;
-
/**
* Accounting of sequence ids per region and then by column family. So we can keep our accounting
* current, call startCacheFlush and then finishedCacheFlush or abortCacheFlush so this instance can
@@ -239,7 +237,6 @@ class SequenceIdAccounting {
}
}
- @VisibleForTesting
ConcurrentMap getOrCreateLowestSequenceIds(byte[] encodedRegionName) {
// Intentionally, this access is done outside of this.regionSequenceIdLock. Done per append.
return computeIfAbsent(this.lowestUnflushedSequenceIds, encodedRegionName,
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
index 3cde0d5113a..a998a3c196a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java
@@ -23,14 +23,13 @@ import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.UUID;
-
import org.apache.hadoop.hbase.zookeeper.ZKListener;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.AuthFailedException;
import org.apache.zookeeper.KeeperException.ConnectionLossException;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java
index 3271696e110..f06b29ccdef 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java
@@ -34,7 +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;
import org.apache.hbase.thirdparty.com.google.common.base.Predicate;
import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
@@ -110,7 +109,7 @@ public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate {
}
}
- @VisibleForTesting
+ @InterfaceAudience.Private
public void setConf(Configuration conf, ZKWatcher zk) {
super.setConf(conf);
try {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
index e51f5ef4979..148b33037cd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
@@ -33,7 +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.hbase.thirdparty.com.google.common.base.Predicate;
import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
@@ -104,7 +103,7 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
}
}
- @VisibleForTesting
+ @InterfaceAudience.Private
public void setConf(Configuration conf, ZKWatcher zk) {
super.setConf(conf);
try {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java
index 4d481239cf0..8d049e99dd8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.replication.ReplicationException;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.ReplicationPeerStorage;
import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
@@ -42,10 +43,6 @@ 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.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
-
/**
* This class is used to upgrade TableCFs from HBase 1.0, 1.1, 1.2, 1.3 to HBase 1.4 or 2.x. It will
* be removed in HBase 3.x. See HBASE-11393
@@ -95,7 +92,6 @@ public class ReplicationPeerConfigUpgrader{
}
}
- @VisibleForTesting
protected String getTableCFsNode(String peerId) {
String replicationZNode = ZNodePaths.joinZNode(zookeeper.getZNodePaths().baseZNode,
conf.get(REPLICATION_ZNODE, REPLICATION_ZNODE_DEFAULT));
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
index c4e1d967b04..6f3c85f97b4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
@@ -39,7 +39,6 @@ import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import java.util.stream.Stream;
-
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
@@ -69,12 +68,11 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
-
-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;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
+
/**
* A {@link org.apache.hadoop.hbase.replication.ReplicationEndpoint}
* implementation for replicating to another HBase cluster.
@@ -308,7 +306,6 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
/**
* Check if there's an {@link TableNotFoundException} in the caused by stacktrace.
*/
- @VisibleForTesting
public static boolean isTableNotFoundException(Throwable io) {
if (io instanceof RemoteException) {
io = ((RemoteException) io).unwrapRemoteException();
@@ -327,7 +324,6 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
/**
* Check if there's an {@link NoSuchColumnFamilyException} in the caused by stacktrace.
*/
- @VisibleForTesting
public static boolean isNoSuchColumnFamilyException(Throwable io) {
if (io instanceof RemoteException) {
io = ((RemoteException) io).unwrapRemoteException();
@@ -343,7 +339,6 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
return false;
}
- @VisibleForTesting
List> filterNotExistTableEdits(final List> oldEntryList) {
List> entryList = new ArrayList<>();
Map existMap = new HashMap<>();
@@ -387,7 +382,6 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
return entryList;
}
- @VisibleForTesting
List> filterNotExistColumnFamilyEdits(final List> oldEntryList) {
List> entryList = new ArrayList<>();
Map> existColumnFamilyMap = new HashMap<>();
@@ -624,7 +618,6 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
notifyStopped();
}
- @VisibleForTesting
protected int replicateEntries(List entries, int batchIndex, int timeout)
throws IOException {
SinkPeer sinkPeer = null;
@@ -679,7 +672,6 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
return batchIndex;
}
- @VisibleForTesting
protected Callable createReplicator(List entries, int batchIndex, int timeout) {
return isSerial ? () -> serialReplicateRegionEntries(entries, batchIndex, timeout)
: () -> replicateEntries(entries, batchIndex, timeout);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
index 3fa3253f2d1..650af2e103a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
@@ -24,15 +24,13 @@ import java.util.Map;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.wal.WAL.Entry;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.metrics.BaseSource;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
/**
* This class is for maintaining the various replication statistics for a source and publishing them
@@ -461,7 +459,7 @@ public class MetricsSource implements BaseSource {
return globalSourceSource.getMetricsName();
}
- @VisibleForTesting
+ @InterfaceAudience.Private
public Map getSingleSourceSourceByTable() {
return singleSourceSourceByTable;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSinkManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSinkManager.java
index 4dc5b334f27..a233196844a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSinkManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSinkManager.java
@@ -27,16 +27,13 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
+import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
+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.Lists;
-import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
-
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
-
/**
* Maintains a collection of peers to replicate to, and randomly selects a
* single peer to replicate to per set of data to replicate. Also handles
@@ -174,7 +171,6 @@ public class ReplicationSinkManager {
return sinks.size();
}
- @VisibleForTesting
protected List getSinksForTesting() {
return Collections.unmodifiableList(sinks);
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
index f30ab29b583..c87ca191138 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
@@ -68,7 +68,7 @@ import org.apache.hadoop.hbase.wal.WAL.Entry;
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;
/**
@@ -143,7 +143,6 @@ public class ReplicationSource implements ReplicationSourceInterface {
private long defaultBandwidth;
private long currentBandwidth;
private WALFileLengthProvider walFileLengthProvider;
- @VisibleForTesting
protected final ConcurrentHashMap workerThreads =
new ConcurrentHashMap<>();
@@ -460,7 +459,6 @@ public class ReplicationSource implements ReplicationSourceInterface {
* Call after {@link #initializeWALEntryFilter(UUID)} else it will be null.
* @return WAL Entry Filter Chain to use on WAL files filtering *out* WALEntry edits.
*/
- @VisibleForTesting
WALEntryFilter getWalEntryFilter() {
return walEntryFilter;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
index 81d68657ec6..05a6aad5bc3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
@@ -72,7 +72,7 @@ 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.util.concurrent.ThreadFactoryBuilder;
/**
@@ -183,7 +183,6 @@ public class ReplicationSourceManager implements ReplicationListener {
* this server (in case it later gets moved back). We synchronize on this instance testing for
* presence and if absent, while creating so only created and started once.
*/
- @VisibleForTesting
AtomicReference catalogReplicationSource = new AtomicReference<>();
/**
@@ -369,7 +368,6 @@ public class ReplicationSourceManager implements ReplicationListener {
* @param peerId the id of the replication peer
* @return the source that was created
*/
- @VisibleForTesting
ReplicationSourceInterface addSource(String peerId) throws IOException {
ReplicationPeer peer = replicationPeers.getPeer(peerId);
ReplicationSourceInterface src = createSource(peerId, peer);
@@ -565,7 +563,6 @@ public class ReplicationSourceManager implements ReplicationListener {
* @param queueId id of the replication queue
* @param queueRecovered Whether this is a recovered queue
*/
- @VisibleForTesting
void cleanOldLogs(String log, boolean inclusive, String queueId, boolean queueRecovered) {
String logPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(log);
if (queueRecovered) {
@@ -597,7 +594,6 @@ public class ReplicationSourceManager implements ReplicationListener {
}
// public because of we call it in TestReplicationEmptyWALRecovery
- @VisibleForTesting
public void preLogRoll(Path newLog) throws IOException {
String logName = newLog.getName();
String logPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(logName);
@@ -655,7 +651,6 @@ public class ReplicationSourceManager implements ReplicationListener {
}
// public because of we call it in TestReplicationEmptyWALRecovery
- @VisibleForTesting
public void postLogRoll(Path newLog) throws IOException {
// This only updates the sources we own, not the recovered ones
for (ReplicationSourceInterface source : this.sources.values()) {
@@ -707,7 +702,6 @@ public class ReplicationSourceManager implements ReplicationListener {
// the rs will abort (See HBASE-20475).
private final Map peersSnapshot;
- @VisibleForTesting
public NodeFailoverWorker(ServerName deadRS) {
super("Failover-for-" + deadRS);
this.deadRS = deadRS;
@@ -845,7 +839,6 @@ public class ReplicationSourceManager implements ReplicationListener {
* Get a copy of the wals of the normal sources on this rs
* @return a sorted set of wal names
*/
- @VisibleForTesting
public Map>> getWALs() {
return Collections.unmodifiableMap(walsById);
}
@@ -854,7 +847,6 @@ public class ReplicationSourceManager implements ReplicationListener {
* Get a copy of the wals of the recovered sources on this rs
* @return a sorted set of wal names
*/
- @VisibleForTesting
Map>> getWalsByIdRecoveredQueues() {
return Collections.unmodifiableMap(walsByIdRecoveredQueues);
}
@@ -879,12 +871,10 @@ public class ReplicationSourceManager implements ReplicationListener {
* Get the normal source for a given peer
* @return the normal source for the give peer if it exists, otherwise null.
*/
- @VisibleForTesting
public ReplicationSourceInterface getSource(String peerId) {
return this.sources.get(peerId);
}
- @VisibleForTesting
List getAllQueues() throws IOException {
List allQueues = Collections.emptyList();
try {
@@ -895,14 +885,12 @@ public class ReplicationSourceManager implements ReplicationListener {
return allQueues;
}
- @VisibleForTesting
int getSizeOfLatestPath() {
synchronized (latestPaths) {
return latestPaths.size();
}
}
- @VisibleForTesting
public AtomicLong getTotalBufferUsed() {
return totalBufferUsed;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALActionListener.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALActionListener.java
index d25ab072eb1..a38a96a3816 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALActionListener.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALActionListener.java
@@ -27,8 +27,6 @@ import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hbase.wal.WALKeyImpl;
import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
/**
* Used to receive new wals.
*/
@@ -65,7 +63,6 @@ class ReplicationSourceWALActionListener implements WALActionsListener {
* @param logKey Key that may get scoped according to its edits
* @param logEdit Edits used to lookup the scopes
*/
- @VisibleForTesting
static void scopeWALEdits(WALKey logKey, WALEdit logEdit, Configuration conf) {
// For bulk load replication we need meta family to know the file we want to replicate.
if (ReplicationUtils.isReplicationForBulkLoadDataEnabled(conf)) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/PermissionStorage.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/PermissionStorage.java
index 2a9b195f0f8..eab3861d7ae 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/PermissionStorage.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/PermissionStorage.java
@@ -30,7 +30,6 @@ import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import java.util.TreeSet;
-
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.AuthUtil;
@@ -73,7 +72,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;
@@ -501,7 +499,6 @@ public final class PermissionStorage {
null, false);
}
- @VisibleForTesting
public static ListMultimap getNamespacePermissions(Configuration conf,
String namespace) throws IOException {
return getPermissions(conf, Bytes.toBytes(toNamespaceEntry(namespace)), null, null, null, null,
@@ -780,7 +777,6 @@ public final class PermissionStorage {
return list;
}
- @VisibleForTesting
public static ListMultimap readUserPermission(byte[] data,
Configuration conf) throws DeserializationException {
if (ProtobufUtil.isPBMagicPrefix(data)) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SnapshotScannerHDFSAclCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SnapshotScannerHDFSAclCleaner.java
index 8ce4266748c..72da07cee5e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SnapshotScannerHDFSAclCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SnapshotScannerHDFSAclCleaner.java
@@ -25,7 +25,6 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate;
@@ -34,8 +33,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;
-
/**
* Implementation of a file cleaner that checks if a empty directory with no subdirs and subfiles is
* deletable when user scan snapshot feature is enabled
@@ -98,7 +95,7 @@ public class SnapshotScannerHDFSAclCleaner extends BaseHFileCleanerDelegate {
}
}
- @VisibleForTesting
+ @InterfaceAudience.Private
static boolean isArchiveDataDir(Path path) {
if (path != null && path.getName().equals(HConstants.BASE_NAMESPACE_DIR)) {
Path parent = path.getParent();
@@ -107,12 +104,12 @@ public class SnapshotScannerHDFSAclCleaner extends BaseHFileCleanerDelegate {
return false;
}
- @VisibleForTesting
+ @InterfaceAudience.Private
static boolean isArchiveNamespaceDir(Path path) {
return path != null && isArchiveDataDir(path.getParent());
}
- @VisibleForTesting
+ @InterfaceAudience.Private
static boolean isArchiveTableDir(Path path) {
return path != null && isArchiveNamespaceDir(path.getParent());
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SnapshotScannerHDFSAclController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SnapshotScannerHDFSAclController.java
index 5c4ba0d6850..4bcf8c608ba 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SnapshotScannerHDFSAclController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SnapshotScannerHDFSAclController.java
@@ -26,7 +26,6 @@ import java.util.List;
import java.util.Optional;
import java.util.Set;
import java.util.stream.Collectors;
-
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
@@ -65,7 +64,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.Sets;
/**
@@ -543,7 +541,7 @@ public class SnapshotScannerHDFSAclController implements MasterCoprocessor, Mast
return isSet;
}
- @VisibleForTesting
+ @InterfaceAudience.Private
boolean checkInitialized(String operation) {
if (initialized) {
if (aclTableInitialized) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/ZKSecretWatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/ZKSecretWatcher.java
index 50f87843903..0fd2205d5a0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/ZKSecretWatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/ZKSecretWatcher.java
@@ -18,19 +18,16 @@
package org.apache.hadoop.hbase.security.token;
-import org.apache.hadoop.hbase.log.HBaseMarkers;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
import java.io.IOException;
import java.util.List;
-
-import org.apache.hadoop.hbase.zookeeper.ZKListener;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.log.HBaseMarkers;
import org.apache.hadoop.hbase.util.Writables;
+import org.apache.hadoop.hbase.zookeeper.ZKListener;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+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.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -235,7 +232,6 @@ public class ZKSecretWatcher extends ZKListener {
* get token keys parent node
* @return token keys parent node
*/
- @VisibleForTesting
String getKeysParentZNode() {
return keysParentZNode;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
index b01466592e6..79e587ac44b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
@@ -51,12 +51,11 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
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.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.protobuf.CodedInputStream;
import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
@@ -197,7 +196,6 @@ public final class SnapshotManifest {
addMobRegion(regionInfo, visitor);
}
- @VisibleForTesting
protected void addMobRegion(RegionInfo regionInfo, RegionVisitor visitor) throws IOException {
// 1. dump region meta info into the snapshot directory
final String snapshotName = desc.getName();
@@ -245,7 +243,6 @@ public final class SnapshotManifest {
addRegion(region, visitor);
}
- @VisibleForTesting
protected void addRegion(final HRegion region, RegionVisitor visitor) throws IOException {
// 1. dump region meta info into the snapshot directory
final String snapshotName = desc.getName();
@@ -294,7 +291,6 @@ public final class SnapshotManifest {
addRegion(tableDir, regionInfo, visitor);
}
- @VisibleForTesting
protected void addRegion(final Path tableDir, final RegionInfo regionInfo, RegionVisitor visitor)
throws IOException {
boolean isMobRegion = MobUtils.isMobRegionInfo(regionInfo);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
index 59abb3c920a..be67d43acfd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
@@ -23,7 +23,6 @@ package org.apache.hadoop.hbase.tool;
import java.util.Map;
import java.util.concurrent.ExecutorService;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.yetus.audience.InterfaceAudience;
@InterfaceAudience.Public
@@ -33,7 +32,7 @@ public interface Canary {
return new CanaryTool(conf, executor);
}
- @VisibleForTesting
+ @InterfaceAudience.Private
static Canary create(Configuration conf, ExecutorService executor, CanaryTool.Sink sink) {
return new CanaryTool(conf, executor, sink);
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/CanaryTool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/CanaryTool.java
index 64a2d5103f8..c4d44ecb234 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/CanaryTool.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/CanaryTool.java
@@ -99,7 +99,7 @@ import org.apache.zookeeper.client.ConnectStringParser;
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.com.google.common.collect.Lists;
/**
@@ -791,7 +791,7 @@ public class CanaryTool implements Tool, Canary {
this(executor, null);
}
- @VisibleForTesting
+ @InterfaceAudience.Private
CanaryTool(ExecutorService executor, Sink sink) {
this.executor = executor;
this.sink = sink;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java
index 866faf223c1..c7ede20963c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java
@@ -97,17 +97,15 @@ import org.apache.hadoop.hbase.util.FSVisitor;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.collect.HashMultimap;
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.Multimap;
import org.apache.hbase.thirdparty.com.google.common.collect.Multimaps;
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;
/**
* Tool to load the output of HFileOutputFormat into an existing table.
@@ -485,7 +483,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
*
* protected for testing.
*/
- @VisibleForTesting
+ @InterfaceAudience.Private
protected void bulkLoadPhase(Table table, Connection conn, ExecutorService pool,
Deque queue, Multimap regionGroups,
boolean copyFile, Map item2RegionMap) throws IOException {
@@ -538,7 +536,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
}
}
- @VisibleForTesting
+ @InterfaceAudience.Private
protected ClientServiceCallable buildClientServiceCallable(Connection conn,
TableName tableName, byte[] first, Collection lqis, boolean copyFile) {
List> famPaths =
@@ -769,7 +767,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
* protected for testing
* @throws IOException if an IO failure is encountered
*/
- @VisibleForTesting
+ @InterfaceAudience.Private
protected Pair, String> groupOrSplit(
Multimap regionGroups, final LoadQueueItem item, final Table table,
final Pair startEndKeys) throws IOException {
@@ -832,7 +830,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
* @deprecated as of release 2.3.0. Use {@link BulkLoadHFiles} instead.
*/
@Deprecated
- @VisibleForTesting
+ @InterfaceAudience.Private
protected List tryAtomicRegionLoad(final Connection conn,
final TableName tableName, final byte[] first, final Collection lqis,
boolean copyFile) throws IOException {
@@ -853,7 +851,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
* @deprecated as of release 2.3.0. Use {@link BulkLoadHFiles} instead.
*/
@Deprecated
- @VisibleForTesting
+ @InterfaceAudience.Private
protected List tryAtomicRegionLoad(ClientServiceCallable serviceCallable,
final TableName tableName, final byte[] first, final Collection lqis)
throws IOException {
@@ -1135,7 +1133,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
* Split a storefile into a top and bottom half, maintaining the metadata, recreating bloom
* filters, etc.
*/
- @VisibleForTesting
+ @InterfaceAudience.Private
static void splitStoreFile(Configuration conf, Path inFile, ColumnFamilyDescriptor familyDesc,
byte[] splitKey, Path bottomOut, Path topOut) throws IOException {
// Open reader with no block cache, and not in-memory
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/coprocessor/CoprocessorValidator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/coprocessor/CoprocessorValidator.java
index 0e76f0ed8c2..766224e5d38 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/coprocessor/CoprocessorValidator.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/coprocessor/CoprocessorValidator.java
@@ -37,7 +37,6 @@ import java.util.Optional;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import java.util.stream.Stream;
-
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.client.Admin;
@@ -53,7 +52,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.cli.CommandLine;
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
@@ -166,7 +164,7 @@ public class CoprocessorValidator extends AbstractHBaseTool {
validateClasses(classLoader, Arrays.asList(classNames), violations);
}
- @VisibleForTesting
+ @InterfaceAudience.Private
protected void validateTables(ClassLoader classLoader, Admin admin,
Pattern pattern, List violations) throws IOException {
List tableDescriptors = admin.listTableDescriptors(pattern);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterUtil.java
index b932c4d9237..c7afb0e5f91 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterUtil.java
@@ -29,8 +29,6 @@ import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
/**
* Utility methods related to BloomFilters
*/
@@ -91,7 +89,6 @@ public final class BloomFilterUtil {
* This gets used in {@link #contains(ByteBuff, int, int, Hash, int, HashKey)}
* @param random The random number source to use, or null to compute actual hashes
*/
- @VisibleForTesting
public static void setRandomGeneratorForTest(Random random) {
randomGeneratorForTest = random;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
index be4c3021608..76a9328b9e5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
@@ -50,7 +50,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.primitives.Ints;
/**
@@ -80,9 +79,7 @@ public class FSTableDescriptors implements TableDescriptors {
private final boolean usecache;
private volatile boolean fsvisited;
- @VisibleForTesting
long cachehits = 0;
- @VisibleForTesting
long invocations = 0;
/**
@@ -117,7 +114,6 @@ public class FSTableDescriptors implements TableDescriptors {
this.usecache = usecache;
}
- @VisibleForTesting
public static void tryUpdateMetaTableDescriptor(Configuration conf) throws IOException {
tryUpdateAndGetMetaTableDescriptor(conf, CommonFSUtils.getCurrentFileSystem(conf),
CommonFSUtils.getRootDir(conf));
@@ -143,7 +139,6 @@ public class FSTableDescriptors implements TableDescriptors {
}
}
- @VisibleForTesting
public static TableDescriptorBuilder createMetaTableDescriptorBuilder(final Configuration conf)
throws IOException {
// TODO We used to set CacheDataInL1 for META table. When we have BucketCache in file mode, now
@@ -183,7 +178,6 @@ public class FSTableDescriptors implements TableDescriptors {
.build());
}
- @VisibleForTesting
protected boolean isUsecache() {
return this.usecache;
}
@@ -290,7 +284,6 @@ public class FSTableDescriptors implements TableDescriptors {
}
}
- @VisibleForTesting
Path updateTableDescriptor(TableDescriptor td) throws IOException {
TableName tableName = td.getTableName();
Path tableDir = getTableDir(tableName);
@@ -401,7 +394,6 @@ public class FSTableDescriptors implements TableDescriptors {
/**
* Compare {@link FileStatus} instances by {@link Path#getName()}. Returns in reverse order.
*/
- @VisibleForTesting
static final Comparator TABLEINFO_FILESTATUS_COMPARATOR =
new Comparator() {
@Override
@@ -413,7 +405,6 @@ public class FSTableDescriptors implements TableDescriptors {
/**
* Return the table directory in HDFS
*/
- @VisibleForTesting
Path getTableDir(final TableName tableName) {
return CommonFSUtils.getTableDir(rootdir, tableName);
}
@@ -428,7 +419,6 @@ public class FSTableDescriptors implements TableDescriptors {
/**
* Width of the sequenceid that is a suffix on a tableinfo file.
*/
- @VisibleForTesting
static final int WIDTH_OF_SEQUENCE_ID = 10;
/**
@@ -458,7 +448,6 @@ public class FSTableDescriptors implements TableDescriptors {
* @param p Path to a .tableinfo file.
* @return The current editid or 0 if none found.
*/
- @VisibleForTesting
static int getTableInfoSequenceId(final Path p) {
if (p == null) {
return 0;
@@ -478,7 +467,6 @@ public class FSTableDescriptors implements TableDescriptors {
* @param sequenceid
* @return Name of tableinfo file.
*/
- @VisibleForTesting
static String getTableInfoFileName(final int sequenceid) {
return TABLEINFO_FILE_PREFIX + "." + formatTableInfoSequenceId(sequenceid);
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
index 19b122f36b1..402c6612bb4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
@@ -84,12 +84,11 @@ import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.StringUtils;
-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.Throwables;
import org.apache.hbase.thirdparty.com.google.common.collect.Iterators;
import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
@@ -109,7 +108,7 @@ public final class FSUtils {
private static final int DEFAULT_THREAD_POOLSIZE = 2;
/** Set to true on Windows platforms */
- @VisibleForTesting // currently only used in testing. TODO refactor into a test class
+ // currently only used in testing. TODO refactor into a test class
public static final boolean WINDOWS = System.getProperty("os.name").startsWith("Windows");
private FSUtils() {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index 6d763c406cb..8c7a74146fd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -117,6 +117,8 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
import org.apache.hadoop.hbase.util.Bytes.ByteArrayComparator;
import org.apache.hadoop.hbase.util.HbckErrorReporter.ERROR_CODE;
import org.apache.hadoop.hbase.util.hbck.HFileCorruptionChecker;
@@ -133,6 +135,10 @@ import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
+import org.apache.hbase.thirdparty.com.google.common.base.Joiner;
+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.com.google.common.collect.Sets;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
@@ -140,15 +146,6 @@ 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.Joiner;
-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.com.google.common.collect.Sets;
-
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
-
/**
* HBaseFsck (hbck) is a tool for checking and repairing region consistency and
* table integrity problems in a corrupted HBase. This tool was written for hbase-1.x. It does not
@@ -215,8 +212,9 @@ public class HBaseFsck extends Configured implements Closeable {
* Here is where hbase-1.x used to default the lock for hbck1.
* It puts in place a lock when it goes to write/make changes.
*/
- @VisibleForTesting
+ @InterfaceAudience.Private
public static final String HBCK_LOCK_FILE = "hbase-hbck.lock";
+
private static final int DEFAULT_MAX_LOCK_FILE_ATTEMPTS = 5;
private static final int DEFAULT_LOCK_FILE_ATTEMPT_SLEEP_INTERVAL = 200; // milliseconds
private static final int DEFAULT_LOCK_FILE_ATTEMPT_MAX_SLEEP_TIME = 5000; // milliseconds
@@ -402,7 +400,7 @@ public class HBaseFsck extends Configured implements Closeable {
/**
* @return Return the tmp dir this tool writes too.
*/
- @VisibleForTesting
+ @InterfaceAudience.Private
public static Path getTmpDir(Configuration conf) throws IOException {
return new Path(CommonFSUtils.getRootDir(conf), HConstants.HBASE_TEMP_DIRECTORY);
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/IdReadWriteLock.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/IdReadWriteLock.java
index 852f5c0dd32..a644a9a34b3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/IdReadWriteLock.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/IdReadWriteLock.java
@@ -23,8 +23,6 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
/**
* Allows multiple concurrent clients to lock on a numeric id with ReentrantReadWriteLock. The
* intended usage for read lock is as follows:
@@ -99,7 +97,6 @@ public class IdReadWriteLock {
}
/** For testing */
- @VisibleForTesting
int purgeAndGetEntryPoolSize() {
gc();
Threads.sleep(200);
@@ -112,7 +109,6 @@ public class IdReadWriteLock {
System.gc();
}
- @VisibleForTesting
public void waitForWaiters(T id, int numWaiters) throws InterruptedException {
for (ReentrantReadWriteLock readWriteLock;;) {
readWriteLock = lockPool.get(id);
@@ -127,7 +123,6 @@ public class IdReadWriteLock {
}
}
- @VisibleForTesting
public ReferenceType getReferenceType() {
return this.refType;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/LossyCounting.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/LossyCounting.java
index 9d7cb566c65..9ade12d578c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/LossyCounting.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/LossyCounting.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.HConstants;
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;
/**
@@ -115,7 +115,6 @@ public class LossyCounting {
/**
* sweep low frequency data
*/
- @VisibleForTesting
public void sweep() {
for(Map.Entry entry : data.entrySet()) {
if(entry.getValue() < currentTerm) {
@@ -168,7 +167,7 @@ public class LossyCounting {
}
}
- @VisibleForTesting public Future> getSweepFuture() {
+ public Future> getSweepFuture() {
return fut.get();
}
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java
index bc6fe9d474a..5822222aa66 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java
@@ -63,7 +63,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.cli.CommandLine;
import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
@@ -138,7 +137,7 @@ public class RegionMover extends AbstractHBaseTool implements Closeable {
private String excludeFile = null;
private String designatedFile = null;
private String defaultDir = System.getProperty("java.io.tmpdir");
- @VisibleForTesting
+ @InterfaceAudience.Private
final int port;
private final Configuration conf;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/ClusterCompactionQueues.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/ClusterCompactionQueues.java
index c0d34d9397e..9be182d245f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/ClusterCompactionQueues.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/ClusterCompactionQueues.java
@@ -25,7 +25,7 @@ import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.hadoop.hbase.ServerName;
import org.apache.yetus.audience.InterfaceAudience;
-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;
@@ -82,7 +82,7 @@ class ClusterCompactionQueues {
}
}
- @VisibleForTesting List getQueue(ServerName serverName) {
+ List getQueue(ServerName serverName) {
lock.readLock().lock();
try {
return compactionQueues.get(serverName);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionRequest.java
index aecfc37d8ef..22ec6cb89ec 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionRequest.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionRequest.java
@@ -38,7 +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;
import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
@InterfaceAudience.Private
@@ -55,7 +54,6 @@ class MajorCompactionRequest {
this.region = region;
}
- @VisibleForTesting
MajorCompactionRequest(Configuration configuration, RegionInfo region,
Set stores) {
this(configuration, region);
@@ -81,7 +79,6 @@ class MajorCompactionRequest {
this.stores = stores;
}
- @VisibleForTesting
Optional createRequest(Configuration configuration,
Set stores, long timestamp) throws IOException {
Set familiesToCompact = getStoresRequiringCompaction(stores, timestamp);
@@ -145,7 +142,6 @@ class MajorCompactionRequest {
return false;
}
- @VisibleForTesting
Connection getConnection(Configuration configuration) throws IOException {
return ConnectionFactory.createConnection(configuration);
}
@@ -166,13 +162,11 @@ class MajorCompactionRequest {
}
- @VisibleForTesting
List | | | |