HBASE-24640 Purge use of VisibleForTesting (#2695)

Signed-off-by: Reid Chan <reidchan@apache.org>
Signed-off-by: Nick Dimiduk <ndimiduk@apache.org>
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupHFileCleaner.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupHFileCleaner.java
index 68e6b43..0550f9b 100644
--- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupHFileCleaner.java
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupHFileCleaner.java
@@ -39,7 +39,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
 
 /**
@@ -88,7 +87,7 @@
     }
   }
 
-  @VisibleForTesting
+  @InterfaceAudience.Private
   void setCheckForFullyBackedUpTables(boolean b) {
     checkForFullyBackedUpTables = b;
   }
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
index d49aef2..e1fb73a 100644
--- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
@@ -52,8 +52,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 /**
  * Handles backup requests, creates backup info records in backup system table to keep track of
  * backup sessions, dispatches backup request.
@@ -101,7 +99,6 @@
    * (TESTs only)
    * @param conf configuration
    */
-  @VisibleForTesting
   public static void decorateMasterConfiguration(Configuration conf) {
     if (!isBackupEnabled(conf)) {
       return;
@@ -137,7 +134,6 @@
    * TESTs only.
    * @param conf configuration
    */
-  @VisibleForTesting
   public static void decorateRegionServerConfiguration(Configuration conf) {
     if (!isBackupEnabled(conf)) {
       return;
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java
index 83e545c..0213414 100644
--- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java
@@ -42,8 +42,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 /**
  * Base class for backup operation. Concrete implementation for
  * full and incremental backup are delegated to corresponding sub-classes:
@@ -55,7 +53,6 @@
 
   public static final String BACKUP_CLIENT_IMPL_CLASS = "backup.client.impl.class";
 
-  @VisibleForTesting
   public static final String BACKUP_TEST_MODE_STAGE = "backup.test.mode.stage";
 
   private static final Logger LOG = LoggerFactory.getLogger(TableBackupClient.class);
@@ -411,12 +408,10 @@
    */
   public abstract void execute() throws IOException;
 
-  @VisibleForTesting
   protected Stage getTestStage() {
     return Stage.valueOf("stage_"+ conf.getInt(BACKUP_TEST_MODE_STAGE, 0));
   }
 
-  @VisibleForTesting
   protected void failStageIf(Stage stage) throws IOException {
     Stage current = getTestStage();
     if (current == stage) {
diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
index 7ec2a22..c987b49 100644
--- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
+++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
@@ -56,8 +56,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 /**
  * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
  * to <code>hbase:meta</code>.
@@ -262,7 +260,6 @@
    *          true and we'll leave out offlined regions from returned list
    * @return List of all user-space regions.
    */
-  @VisibleForTesting
   public static List<RegionInfo> getAllRegions(Connection connection,
     boolean excludeOfflinedSplitParents) throws IOException {
     List<Pair<RegionInfo, ServerName>> result;
@@ -861,7 +858,6 @@
    * @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);
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/CatalogFamilyFormat.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/CatalogFamilyFormat.java
index 1633707..3cf6cc0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/CatalogFamilyFormat.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/CatalogFamilyFormat.java
@@ -39,8 +39,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 /**
  * Helper class for generating/parsing
  * {@value org.apache.hadoop.hbase.HConstants#CATALOG_FAMILY_STR} family cells in meta table.
@@ -276,7 +274,6 @@
   }
 
   /** The delimiter for meta columns for replicaIds &gt; 0 */
-  @VisibleForTesting
   static final char META_REPLICA_ID_DELIMITER = '_';
 
   /**
@@ -285,7 +282,6 @@
    * @param serverColumn the column qualifier
    * @return an int for the replicaId
    */
-  @VisibleForTesting
   static int parseReplicaIdFromServerColumn(byte[] serverColumn) {
     String serverStr = Bytes.toString(serverColumn);
 
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 7aa9597..fcd1724 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
@@ -32,7 +32,6 @@
 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;
 
@@ -60,7 +59,6 @@
 
   private boolean closed;
 
-  @VisibleForTesting
   Timeout periodicFlushTask;
 
   AsyncBufferedMutatorImpl(HashedWheelTimer periodicalFlushTimer, AsyncTable<?> table,
@@ -83,7 +81,6 @@
   }
 
   // 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 fda262c..51b84c0 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
@@ -59,7 +59,6 @@
 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;
@@ -75,7 +74,6 @@
 
   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(),
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java
index 3571f960..5ae9de6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java
@@ -30,8 +30,6 @@
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.yetus.audience.InterfaceAudience;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 /**
  * The asynchronous locator for meta region.
  */
@@ -136,13 +134,11 @@
   }
 
   // only used for testing whether we have cached the location for a region.
-  @VisibleForTesting
   RegionLocations getRegionLocationInCache() {
     return metaRegionLocations.get();
   }
 
   // only used for testing whether we have cached the location for a table.
-  @VisibleForTesting
   int getNumberOfCachedRegionLocations() {
     RegionLocations locs = metaRegionLocations.get();
     return locs != null ? locs.numNonNullElements() : 0;
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
index a9ee6a9..2c2520f 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
@@ -63,7 +63,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.base.Objects;
 
 /**
@@ -74,13 +73,11 @@
 
   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;
@@ -709,7 +706,6 @@
   }
 
   // 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) {
@@ -719,7 +715,6 @@
   }
 
   // only used for testing whether we have cached the location for a table.
-  @VisibleForTesting
   int getNumberOfCachedRegionLocations(TableName tableName) {
     TableCache tableCache = cache.get(tableName);
     if (tableCache == null) {
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 09eabfc..215a1c5 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
@@ -34,7 +34,6 @@
 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;
 
@@ -175,13 +174,11 @@
     nonMetaRegionLocator.clearCache();
   }
 
-  @VisibleForTesting
   AsyncNonMetaRegionLocator getNonMetaRegionLocator() {
     return nonMetaRegionLocator;
   }
 
   // only used for testing whether we have cached the location for a region.
-  @VisibleForTesting
   RegionLocations getRegionLocationInCache(TableName tableName, byte[] row) {
     if (TableName.isMetaTableName(tableName)) {
       return metaRegionLocator.getRegionLocationInCache();
@@ -191,7 +188,6 @@
   }
 
   // only used for testing whether we have cached the location for a table.
-  @VisibleForTesting
   int getNumberOfCachedRegionLocations(TableName tableName) {
     if (TableName.isMetaTableName(tableName)) {
       return metaRegionLocator.getNumberOfCachedRegionLocations();
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 7fe6d12..2858d2f 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.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 @@
   }
 
   // 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/ConnectionConfiguration.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java
index 9edf8c2..19ca9ad 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.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.
@@ -125,7 +123,6 @@
    * 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/MasterRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
index 0658268..0975289 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
@@ -49,7 +49,6 @@
 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;
@@ -151,7 +150,6 @@
    * <p/>
    * Will be called in {@code HBaseTestingUtility}.
    */
-  @VisibleForTesting
   public static String getMasterAddr(Configuration conf) throws UnknownHostException {
     String masterAddrFromConf = conf.get(MASTER_ADDRS_KEY);
     if (!Strings.isNullOrEmpty(masterAddrFromConf)) {
@@ -331,7 +329,6 @@
             "getMasters()").thenApply(MasterRegistry::transformServerNames);
   }
 
-  @VisibleForTesting
   Set<ServerName> 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 e9f4c61..f2e4218 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.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;
@@ -126,12 +125,11 @@
     }
   }
 
-  @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 +180,6 @@
     }
   }
 
-  @VisibleForTesting
   protected static class RunnerStats {
     final Counter normalRunners;
     final Counter delayRunners;
@@ -210,7 +207,6 @@
     }
   }
 
-  @VisibleForTesting
   protected ConcurrentHashMap<ServerName, ConcurrentMap<byte[], RegionStats>> serverStats
           = new ConcurrentHashMap<>();
 
@@ -275,36 +271,36 @@
 
   // static metrics
 
-  @VisibleForTesting protected final Counter metaCacheHits;
-  @VisibleForTesting protected final Counter metaCacheMisses;
-  @VisibleForTesting protected final CallTracker getTracker;
-  @VisibleForTesting protected final CallTracker scanTracker;
-  @VisibleForTesting protected final CallTracker appendTracker;
-  @VisibleForTesting protected final CallTracker deleteTracker;
-  @VisibleForTesting protected final CallTracker incrementTracker;
-  @VisibleForTesting protected final CallTracker putTracker;
-  @VisibleForTesting protected final CallTracker multiTracker;
-  @VisibleForTesting protected final RunnerStats runnerStats;
-  @VisibleForTesting protected final Counter metaCacheNumClearServer;
-  @VisibleForTesting protected final Counter metaCacheNumClearRegion;
-  @VisibleForTesting protected final Counter hedgedReadOps;
-  @VisibleForTesting protected final Counter hedgedReadWin;
-  @VisibleForTesting protected final Histogram concurrentCallsPerServerHist;
-  @VisibleForTesting protected final Histogram numActionsPerServerHist;
+  protected final Counter metaCacheHits;
+  protected final Counter metaCacheMisses;
+  protected final CallTracker getTracker;
+  protected final CallTracker scanTracker;
+  protected final CallTracker appendTracker;
+  protected final CallTracker deleteTracker;
+  protected final CallTracker incrementTracker;
+  protected final CallTracker putTracker;
+  protected final CallTracker multiTracker;
+  protected final RunnerStats runnerStats;
+  protected final Counter metaCacheNumClearServer;
+  protected final Counter metaCacheNumClearRegion;
+  protected final Counter hedgedReadOps;
+  protected final Counter hedgedReadWin;
+  protected final Histogram concurrentCallsPerServerHist;
+  protected final Histogram numActionsPerServerHist;
 
   // dynamic metrics
 
   // These maps are used to cache references to the metric instances that are managed by the
   // registry. I don't think their use perfectly removes redundant allocations, but it's
   // a big improvement over calling registry.newMetric each time.
-  @VisibleForTesting protected final ConcurrentMap<String, Timer> rpcTimers =
+  protected final ConcurrentMap<String, Timer> rpcTimers =
       new ConcurrentHashMap<>(CAPACITY, LOAD_FACTOR, CONCURRENCY_LEVEL);
-  @VisibleForTesting protected final ConcurrentMap<String, Histogram> rpcHistograms =
+  protected final ConcurrentMap<String, Histogram> rpcHistograms =
       new ConcurrentHashMap<>(CAPACITY * 2 /* tracking both request and response sizes */,
           LOAD_FACTOR, CONCURRENCY_LEVEL);
   private final ConcurrentMap<String, Counter> cacheDroppingExceptions =
     new ConcurrentHashMap<>(CAPACITY, LOAD_FACTOR, CONCURRENCY_LEVEL);
-  @VisibleForTesting protected final ConcurrentMap<String, Counter>  rpcCounters =
+  protected final ConcurrentMap<String, Counter>  rpcCounters =
       new ConcurrentHashMap<>(CAPACITY, LOAD_FACTOR, CONCURRENCY_LEVEL);
 
   MetricsConnection(String scope, Supplier<ThreadPoolExecutor> batchPool,
@@ -358,17 +354,14 @@
     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/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index 6287952..512e7a9 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
@@ -100,7 +100,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hbase.thirdparty.com.google.protobuf.Message;
 import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
@@ -2385,7 +2384,6 @@
    * @param regionNameOrEncodedRegionName region name or encoded region name
    * @return region location, wrapped by a {@link CompletableFuture}
    */
-  @VisibleForTesting
   CompletableFuture<HRegionLocation> getRegionLocation(byte[] regionNameOrEncodedRegionName) {
     if (regionNameOrEncodedRegionName == null) {
       return failedFuture(new IllegalArgumentException("Passed region name can't be null"));
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 d870144..1a184da 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,6 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -63,7 +62,6 @@
   /**
    * Default value of {@link #HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE}.
    */
-  @VisibleForTesting
   static final long DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE = 4194304;
 
   /**
@@ -73,7 +71,6 @@
   /**
    * Default value of {@link #HBASE_CLIENT_MAX_PERREQUEST_ROWS}.
    */
-  @VisibleForTesting
   static final long DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_ROWS = 2048;
 
   /**
@@ -83,14 +80,10 @@
   /**
    * Default value of {@link #HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE}.
    */
-  @VisibleForTesting
   static final long DEFAULT_HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE = DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE;
-  @VisibleForTesting
   final AtomicLong tasksInProgress = new AtomicLong(0);
-  @VisibleForTesting
   final ConcurrentMap<byte[], AtomicInteger> taskCounterPerRegion
           = new ConcurrentSkipListMap<>(Bytes.BYTES_COMPARATOR);
-  @VisibleForTesting
   final ConcurrentMap<ServerName, AtomicInteger> taskCounterPerServer = new ConcurrentHashMap<>();
   /**
    * The number of tasks simultaneously executed on the cluster.
@@ -112,13 +105,11 @@
    * 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 =
@@ -171,7 +162,6 @@
     return value;
   }
 
-  @VisibleForTesting
   static Checker newChecker(List<RowChecker> checkers) {
     return new Checker() {
       private boolean isEnd = false;
@@ -331,7 +321,6 @@
    * 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;
@@ -365,7 +354,6 @@
   /**
    * limit the max number of tasks in an AsyncProcess.
    */
-  @VisibleForTesting
   static class TaskCountChecker implements RowChecker {
 
     private static final long MAX_WAITING_TIME = 1000; //ms
@@ -475,7 +463,6 @@
   /**
    * limit the number of rows for each request.
    */
-  @VisibleForTesting
   static class RequestRowsChecker implements RowChecker {
 
     private final long maxRowsPerRequest;
@@ -514,7 +501,6 @@
   /**
    * limit the heap size for each request.
    */
-  @VisibleForTesting
   static class RequestHeapSizeChecker implements RowChecker {
 
     private final long maxHeapSizePerRequest;
@@ -554,7 +540,6 @@
   /**
    * 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 f1f0521..4b31c7a 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.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.List;
 import java.util.concurrent.CompletableFuture;
@@ -42,7 +43,7 @@
 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;
 
@@ -96,7 +97,6 @@
     return getAndConvert(znodePaths.clusterIdZNode, ZKConnectionRegistry::getClusterId);
   }
 
-  @VisibleForTesting
   ReadOnlyZKClient getZKClient() {
     return zk;
   }
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 6b1e251..2482a63 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.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.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 @@
    * For test only. Usually you should use the {@link #isConnectionException(Throwable)} method
    * below.
    */
-  @VisibleForTesting
   public static Set<Class<? extends Throwable>> getConnectionExceptionTypes() {
     return CONNECTION_EXCEPTION_TYPES;
   }
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 f6811f6..9f44fe8 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.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.
@@ -317,12 +315,10 @@
     NO_NEXT
   }
 
-  @VisibleForTesting
   static SatisfiesCode satisfies(byte[] row, byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) {
     return satisfies(false, row, 0, row.length, fuzzyKeyBytes, fuzzyKeyMeta);
   }
 
-  @VisibleForTesting
   static SatisfiesCode satisfies(boolean reverse, byte[] row, byte[] fuzzyKeyBytes,
       byte[] fuzzyKeyMeta) {
     return satisfies(reverse, row, 0, row.length, fuzzyKeyBytes, fuzzyKeyMeta);
@@ -438,12 +434,10 @@
     return SatisfiesCode.YES;
   }
 
-  @VisibleForTesting
   static byte[] getNextForFuzzyRule(byte[] row, byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) {
     return getNextForFuzzyRule(false, row, 0, row.length, fuzzyKeyBytes, fuzzyKeyMeta);
   }
 
-  @VisibleForTesting
   static byte[] getNextForFuzzyRule(boolean reverse, byte[] row, byte[] fuzzyKeyBytes,
       byte[] fuzzyKeyMeta) {
     return getNextForFuzzyRule(reverse, row, 0, row.length, fuzzyKeyBytes, fuzzyKeyMeta);
@@ -530,7 +524,6 @@
    * @return greater byte array than given (row) which satisfies the fuzzy rule if it exists, null
    *         otherwise
    */
-  @VisibleForTesting
   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 acc82de..be125f3 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,16 +44,15 @@
 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.base.Preconditions;
 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;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hbase.thirdparty.com.google.protobuf.BlockingRpcChannel;
 import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
 import org.apache.hbase.thirdparty.com.google.protobuf.Message;
@@ -216,7 +215,6 @@
     }
   }
 
-  @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 @@
   }
 
   // for writing tests that want to throw exception when connecting.
-  @VisibleForTesting
   boolean isTcpNoDelay() {
     return tcpNoDelay;
   }
@@ -565,7 +562,6 @@
   /**
    * 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 22eca53..dd8f96b 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.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 @@
    * 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 811bb2c..1b1411c 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.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 @@
     }
   };
 
-  @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 b6f9e38..4347952 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 @@
   }
 
   /** 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 50594c8..2d03473 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.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 @@
   // 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 3c25d6e..d77d8d1 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 @@
 
 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 @@
   /**
    * Returns a copy of the internal state of <code>this</code>
    */
-  @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 f425984..e00380d 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
@@ -124,7 +124,6 @@
 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;
@@ -290,7 +289,6 @@
     }
   }
 
-  @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 b4f3ccb..a1475de 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.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 @@
 
   private final AtomicBoolean closed = new AtomicBoolean(false);
 
-  @VisibleForTesting
   ZooKeeper zookeeper;
 
   private int pendingRequests = 0;
@@ -365,7 +362,6 @@
     }
   }
 
-  @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 caf11c3..a2a8198 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.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 @@
     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/ChoreService.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java
index 9dbb307..39c3ccc 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.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 @@
    *          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 b7ddbab..856480f 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.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 /**
  * An HBase Key/Value. This is the fundamental HBase Type.
  * <p>
@@ -1232,7 +1230,6 @@
    * 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 2aadc42..a86ca10 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.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.
@@ -2635,7 +2633,6 @@
    * @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 a009cf4..1fb5b7e 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.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 @@
    * 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 @@
     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 @@
   }
 
   @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 2c36c53..ad26f76 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.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
@@ -140,7 +139,6 @@
   /**
    * @return true if contains the observer, for unit test only
    */
-  @VisibleForTesting
   public boolean containsObserver(ConfigurationObserver observer) {
     synchronized (configurationObservers) {
       return configurationObservers.contains(observer);
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 8741bcf..f0cb9b0 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,9 +25,6 @@
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.LongAdder;
-
-import sun.nio.ch.DirectBuffer;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.nio.ByteBuff;
@@ -35,8 +32,8 @@
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import sun.nio.ch.DirectBuffer;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 
 /**
@@ -208,7 +205,6 @@
     return new ByteBuffAllocator(false, 0, DEFAULT_BUFFER_SIZE, Integer.MAX_VALUE);
   }
 
-  @VisibleForTesting
   ByteBuffAllocator(boolean reservoirEnabled, int maxBufCount, int bufSize,
       int minSizeForReservoirUse) {
     this.reservoirEnabled = reservoirEnabled;
@@ -241,7 +237,6 @@
    * 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();
   }
@@ -348,7 +343,6 @@
   /**
    * 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 73b8713..69f12f9 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.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.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 @@
     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 a119c57..19c2bd8 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.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.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 @@
     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 afaf197..0f19254 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.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.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 @@
  * 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 a488185..68627c3 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.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 @@
       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 efa18fb..0c054ce 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.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.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 @@
 
   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 42d1bf4..0c4c52f 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.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 @@
         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 @@
     }
   }
 
-  @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 08c35be..d270d63 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
@@ -27,7 +27,6 @@
 import java.math.BigInteger;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
-
 import org.apache.hadoop.hbase.io.ByteBufferWriter;
 import org.apache.hadoop.hbase.io.util.StreamUtils;
 import org.apache.hadoop.hbase.nio.ByteBuff;
@@ -36,8 +35,6 @@
 import org.apache.yetus.audience.InterfaceAudience;
 import sun.nio.ch.DirectBuffer;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 /**
  * Utility functions for working with byte buffers, such as reading/writing
  * variable-length long numbers.
@@ -49,7 +46,6 @@
   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
   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 6ecfa10..ce24694 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
@@ -37,7 +37,6 @@
 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.io.RawComparator;
@@ -46,10 +45,8 @@
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import sun.misc.Unsafe;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
 
 /**
@@ -127,7 +124,6 @@
   // SizeOf which uses java.lang.instrument says 24 bytes. (3 longs?)
   public static final int ESTIMATED_HEAP_TAX = 16;
 
-  @VisibleForTesting
   static final boolean UNSAFE_UNALIGNED = UnsafeAvailChecker.unaligned();
 
   /**
@@ -1270,7 +1266,6 @@
 
   }
 
-  @VisibleForTesting
   static Comparer<byte[]> lexicographicalComparerJavaImpl() {
     return LexicographicalComparerHolder.PureJavaComparer.INSTANCE;
   }
@@ -1420,7 +1415,6 @@
    * <p>Uses reflection to gracefully fall back to the Java implementation if
    * {@code Unsafe} isn't available.
    */
-  @VisibleForTesting
   static class LexicographicalComparerHolder {
     static final String UNSAFE_COMPARER_NAME =
         LexicographicalComparerHolder.class.getName() + "$UnsafeComparer";
@@ -1470,7 +1464,6 @@
       }
     }
 
-    @VisibleForTesting
     enum UnsafeComparer implements Comparer<byte[]> {
       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 6f88c00..b983fc0 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,7 +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;
 
 
 /**
@@ -236,7 +235,6 @@
   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 5859c3a..fef8e29 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.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.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;
 
 /**
@@ -346,7 +343,7 @@
   }
 
   /**
-   * 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
@@ -359,7 +356,6 @@
     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 9e5692f..112af1e 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.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 @@
     }
   }
 
-  @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 ff2fd45..f565bc1 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 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,6 @@
    * @param comp Compliment the encoded value when {@code comp} is true.
    * @return number of bytes written.
    */
-  @VisibleForTesting
   static int putVaruint64(PositionedByteRange dst, long val, boolean comp) {
     int w, y, len = 0;
     final int offset = dst.getOffset(), start = dst.getPosition();
@@ -457,7 +454,6 @@
    * @param comp if true, parse the compliment of the value.
    * @return the number of bytes consumed by this value.
    */
-  @VisibleForTesting
   static int lengthVaruint64(PositionedByteRange src, boolean comp) {
     int a0 = (comp ? DESCENDING : ASCENDING).apply(src.peek()) & 0xff;
     if (a0 <= 240) return 1;
@@ -478,7 +474,6 @@
    * @param cmp if true, parse the compliment of the value.
    * @return the number of bytes skipped.
    */
-  @VisibleForTesting
   static int skipVaruint64(PositionedByteRange src, boolean cmp) {
     final int len = lengthVaruint64(src, cmp);
     src.setPosition(src.getPosition() + len);
@@ -490,7 +485,6 @@
    * encoded value when {@code comp} is true.
    * @return the decoded value.
    */
-  @VisibleForTesting
   static long getVaruint64(PositionedByteRange src, boolean comp) {
     assert src.getRemaining() >= lengthVaruint64(src, comp);
     final long ret;
@@ -547,7 +541,6 @@
    * From Phoenix's {@code NumberUtil}.
    * @return new {@link BigDecimal} instance
    */
-  @VisibleForTesting
   static BigDecimal normalize(BigDecimal val) {
     return null == val ? null : val.stripTrailingZeros().round(DEFAULT_MATH_CONTEXT);
   }
@@ -1013,7 +1006,6 @@
   /**
    * Calculate the expected BlobVar decoded length based on encoded length.
    */
-  @VisibleForTesting
   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 9882777..e4e4ce8 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
@@ -28,8 +28,6 @@
 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.
  *
@@ -299,7 +297,6 @@
    * @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(",");
@@ -312,7 +309,6 @@
   //      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 16bce3e..c30e324 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.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 @@
    * @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 32ffa7f..79aae9b 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.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 @@
    * @return A running SimpleKdcServer on loopback/'localhost' on a random port
    * @see #getRunningSimpleKdcServer(File, Supplier)
    */
-  @VisibleForTesting
   static SimpleKdcServer getRunningSimpleKdcServer(File testDir,
       Supplier<Integer> randomPortGenerator, final boolean portClash)
         throws KrbException, IOException {
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 a93718f..2b38dcb 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
@@ -71,7 +71,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
 import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
 import org.apache.hbase.thirdparty.com.google.protobuf.Service;
@@ -103,7 +102,7 @@
     System.exit(response == null ? -1 : 0);
   }
 
-  @VisibleForTesting
+  @InterfaceAudience.Private
   static Map<byte[], Response> run(final Configuration conf, final String[] args) throws Throwable {
     String[] otherArgs = new GenericOptionsParser(conf, args).getRemainingArgs();
     if (!ExportUtils.isValidArguements(args)) {
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/metrics/impl/GlobalMetricRegistriesAdapter.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/metrics/impl/GlobalMetricRegistriesAdapter.java
index 96fc954..a816d49 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/metrics/impl/GlobalMetricRegistriesAdapter.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/metrics/impl/GlobalMetricRegistriesAdapter.java
@@ -38,8 +38,6 @@
 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 @@
     return new GlobalMetricRegistriesAdapter();
   }
 
-  @VisibleForTesting
   public void stop() {
     stopped.set(true);
   }
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableLatenciesImpl.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableLatenciesImpl.java
index 5e13a61..dd143d4 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableLatenciesImpl.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableLatenciesImpl.java
@@ -25,8 +25,6 @@
 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 @@
 
   private final HashMap<TableName,TableHistograms> histogramsByTable = new HashMap<>();
 
-  @VisibleForTesting
   public static class TableHistograms {
     final MetricHistogram getTimeHisto;
     final MetricHistogram incrementTimeHisto;
@@ -120,7 +117,6 @@
     }
   }
 
-  @VisibleForTesting
   public static String qualifyMetricsName(TableName tableName, String metric) {
     StringBuilder sb = new StringBuilder();
     sb.append("Namespace_").append(tableName.getNamespaceAsString());
@@ -129,7 +125,6 @@
     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-hadoop-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java
index fbcd9fc..88b491b 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java
@@ -30,8 +30,6 @@
 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 @@
    * 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 @@
    * Restarts the stopped service.
    * @see #stop()
    */
-  @VisibleForTesting
   public static void restart() {
     stopped.set(false);
   }
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/metrics2/util/MetricSampleQuantiles.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/metrics2/util/MetricSampleQuantiles.java
index 8d07558..c1880f8 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/metrics2/util/MetricSampleQuantiles.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/metrics2/util/MetricSampleQuantiles.java
@@ -27,8 +27,6 @@
 
 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 @@
    * 
    * @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 1863557..41844e7 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.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 @@
 
   private final List<ListenerInfo> listeners = Lists.newArrayList();
 
-  @VisibleForTesting
   public List<ServerConnector> getServerConnectors() {
     return listeners.stream().map(info -> info.listener).collect(Collectors.toList());
   }
@@ -1122,7 +1120,6 @@
    * 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 dede1f9..1fcfa13 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.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.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 @@
         protocol.equals(PROTOCOL_HTTPS)));
   }
 
-  @VisibleForTesting
   static class CLI extends Configured implements Tool {
     private Operations operation = Operations.UNKNOWN;
     private String protocol;
@@ -289,7 +286,7 @@
       // 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 da907cb..0484fbb 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
@@ -23,8 +23,6 @@
 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
  * to this main.
@@ -35,7 +33,7 @@
 
   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 a4f092b..568c47f 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
@@ -37,7 +37,6 @@
 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;
@@ -106,7 +105,6 @@
     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 08752c1..75b5246 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.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
@@ -665,7 +663,7 @@
    * @param conf to read the serialized values from
    * @return a map from column family to the configured compression algorithm
    */
-  @VisibleForTesting
+  @InterfaceAudience.Private
   static Map<byte[], Algorithm> createFamilyCompressionMap(Configuration
       conf) {
     Map<byte[], String> stringMap = createFamilyConfValueMap(conf,
@@ -685,7 +683,7 @@
    * @param conf to read the serialized values from
    * @return a map from column family to the the configured bloom filter type
    */
-  @VisibleForTesting
+  @InterfaceAudience.Private
   static Map<byte[], BloomType> createFamilyBloomTypeMap(Configuration conf) {
     Map<byte[], String> stringMap = createFamilyConfValueMap(conf,
         BLOOM_TYPE_FAMILIES_CONF_KEY);
@@ -704,12 +702,11 @@
    * @param conf to read the serialized values from
    * @return a map from column family to the the configured bloom filter param
    */
-  @VisibleForTesting
+  @InterfaceAudience.Private
   static Map<byte[], String> createFamilyBloomParamMap(Configuration conf) {
     return createFamilyConfValueMap(conf, BLOOM_PARAM_FAMILIES_CONF_KEY);
   }
 
-
   /**
    * Runs inside the task to deserialize column family to block size
    * map from the configuration.
@@ -717,7 +714,7 @@
    * @param conf to read the serialized values from
    * @return a map from column family to the configured block size
    */
-  @VisibleForTesting
+  @InterfaceAudience.Private
   static Map<byte[], Integer> createFamilyBlockSizeMap(Configuration conf) {
     Map<byte[], String> stringMap = createFamilyConfValueMap(conf,
         BLOCK_SIZE_FAMILIES_CONF_KEY);
@@ -737,7 +734,7 @@
    * @return a map from column family to HFileDataBlockEncoder for the
    *         configured data block type for the family
    */
-  @VisibleForTesting
+  @InterfaceAudience.Private
   static Map<byte[], DataBlockEncoding> createFamilyDataBlockEncodingMap(
       Configuration conf) {
     Map<byte[], String> stringMap = createFamilyConfValueMap(conf,
@@ -749,7 +746,6 @@
     return encoderMap;
   }
 
-
   /**
    * Run inside the task to deserialize column family to given conf value map.
    *
@@ -802,7 +798,7 @@
 
   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value =
     "RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE")
-  @VisibleForTesting
+  @InterfaceAudience.Private
   static String serializeColumnFamilyAttribute(Function<ColumnFamilyDescriptor, String> fn,
         List<TableDescriptor> allTables)
       throws UnsupportedEncodingException {
@@ -833,7 +829,7 @@
    * Serialize column family to compression algorithm map to configuration.
    * Invoked while configuring the MR job for incremental load.
    */
-  @VisibleForTesting
+  @InterfaceAudience.Private
   static Function<ColumnFamilyDescriptor, String> compressionDetails = familyDescriptor ->
           familyDescriptor.getCompressionType().getName();
 
@@ -841,7 +837,7 @@
    * Serialize column family to block size map to configuration. Invoked while
    * configuring the MR job for incremental load.
    */
-  @VisibleForTesting
+  @InterfaceAudience.Private
   static Function<ColumnFamilyDescriptor, String> blockSizeDetails = familyDescriptor -> String
           .valueOf(familyDescriptor.getBlocksize());
 
@@ -849,7 +845,7 @@
    * Serialize column family to bloom type map to configuration. Invoked while
    * configuring the MR job for incremental load.
    */
-  @VisibleForTesting
+  @InterfaceAudience.Private
   static Function<ColumnFamilyDescriptor, String> bloomTypeDetails = familyDescriptor -> {
     String bloomType = familyDescriptor.getBloomFilterType().toString();
     if (bloomType == null) {
@@ -862,7 +858,7 @@
    * Serialize column family to bloom param map to configuration. Invoked while
    * configuring the MR job for incremental load.
    */
-  @VisibleForTesting
+  @InterfaceAudience.Private
   static Function<ColumnFamilyDescriptor, String> bloomParamDetails = familyDescriptor -> {
     BloomType bloomType = familyDescriptor.getBloomFilterType();
     String bloomParam = "";
@@ -876,7 +872,7 @@
    * Serialize column family to data block encoding map to configuration.
    * Invoked while configuring the MR job for incremental load.
    */
-  @VisibleForTesting
+  @InterfaceAudience.Private
   static Function<ColumnFamilyDescriptor, String> dataBlockEncodingDetails = familyDescriptor -> {
     DataBlockEncoding encoding = familyDescriptor.getDataBlockEncoding();
     if (encoding == null) {
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 cdc2f94..6410bf8 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 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 @@
  *     -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 8baf85f..6679010 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.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 @@
     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 c84b25b..81f9a7c 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
@@ -37,7 +37,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;
 
 /**
  * Iterate over an HBase table data, return (ImmutableBytesWritable, Result)
@@ -51,8 +50,9 @@
   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 3ca6c03..d9c9b87 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
@@ -40,8 +40,6 @@
 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,
@@ -131,7 +129,7 @@
     }
   }
 
-  @VisibleForTesting
+  @InterfaceAudience.Private
   static class TableSnapshotRegionRecordReader extends
       RecordReader<ImmutableBytesWritable, Result> {
     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 7b940d7..baa0d1d 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.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 @@
     scan.withStopRow(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 e17a28a..edc813d 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.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 @@
    * implementation will be loaded.
    * @return A {@link MetricRegistries} implementation.
    */
-  @VisibleForTesting
   static MetricRegistries load(List<MetricRegistries> availableImplementations) {
 
     if (availableImplementations.size() == 1) {
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/AbstractProcedureScheduler.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/AbstractProcedureScheduler.java
index 6b44614..53bfba6 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/AbstractProcedureScheduler.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/AbstractProcedureScheduler.java
@@ -26,8 +26,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 @InterfaceAudience.Private
 public abstract class AbstractProcedureScheduler implements ProcedureScheduler {
   private static final Logger LOG = LoggerFactory.getLogger(AbstractProcedureScheduler.class);
@@ -246,7 +244,6 @@
    * Access should remain package-private. Use ProcedureEvent class to wake/suspend events.
    * @param events the list of events to wake
    */
-  @VisibleForTesting
   public void wakeEvents(ProcedureEvent[] events) {
     schedLock();
     try {
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 46dd9e2..579c609 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.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;
 
 /**
@@ -588,7 +586,6 @@
   /**
    * Called by the ProcedureExecutor to assign the ID to the newly created procedure.
    */
-  @VisibleForTesting
   protected void setProcId(long procId) {
     this.procId = procId;
     this.submittedTime = EnvironmentEdgeManager.currentTime();
@@ -609,12 +606,10 @@
   /**
    * Called by the ProcedureExecutor to set the value to the newly created procedure.
    */
-  @VisibleForTesting
   protected void setNonceKey(NonceKey nonceKey) {
     this.nonceKey = nonceKey;
   }
 
-  @VisibleForTesting
   public void setOwner(String owner) {
     this.owner = StringUtils.isEmpty(owner) ? null : owner;
   }
@@ -784,7 +779,6 @@
     return false;
   }
 
-  @VisibleForTesting
   protected synchronized void setState(final ProcedureState state) {
     this.state = state;
     updateTimestamp();
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureEvent.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureEvent.java
index 2cf30b2..614aeb0 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureEvent.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureEvent.java
@@ -21,7 +21,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;
 
 /**
  * Basic ProcedureEvent that contains an "object", which can be a description or a reference to the
@@ -107,7 +106,6 @@
    * when waking up multiple events.
    * Access should remain package-private.
    */
-  @VisibleForTesting
   public synchronized void wakeInternal(AbstractProcedureScheduler procedureScheduler) {
     if (ready && !suspendedProcedures.isEmpty()) {
       LOG.warn("Found procedures suspended in a ready event! Size=" + suspendedProcedures.size());
@@ -127,7 +125,6 @@
    * 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 b99f544..9111345 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.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 @@
     workerMonitorExecutor.sendStopSignal();
   }
 
-  @VisibleForTesting
   public void join() {
     assert !isRunning() : "expected not running";
 
@@ -1332,12 +1330,10 @@
     return procId;
   }
 
-  @VisibleForTesting
   protected long getLastProcId() {
     return lastProcId.get();
   }
 
-  @VisibleForTesting
   public Set<Long> getActiveProcIds() {
     return procedures.keySet();
   }
@@ -1932,17 +1928,14 @@
     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 d787cc0..72b2b28 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.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 @@
    * Returns the number of elements in this queue.
    * @return the number of elements in this queue.
    */
-  @VisibleForTesting
   int size();
 
   /**
@@ -133,6 +130,5 @@
    * 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 feab8be..2b043d4 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 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 @@
     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 46c4c5e..d1af496 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
@@ -22,14 +22,11 @@
 import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
-
 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.ProcedureProtos.StateMachineProcedureData;
 
 /**
@@ -73,7 +70,6 @@
    */
   private int previousState;
 
-  @VisibleForTesting
   public enum Flow {
     HAS_MORE_STATE,
     NO_MORE_STATE,
@@ -282,7 +278,6 @@
    * 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 b0301c6..6aed228 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.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 @@
       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 @@
     return (System.currentTimeMillis() - lastRollTs.get());
   }
 
-  @VisibleForTesting
   void periodicRollForTesting() throws IOException {
     lock.lock();
     try {
@@ -994,7 +991,6 @@
     }
   }
 
-  @VisibleForTesting
   public boolean rollWriterForTesting() throws IOException {
     lock.lock();
     try {
@@ -1004,7 +1000,6 @@
     }
   }
 
-  @VisibleForTesting
   void removeInactiveLogsForTesting() throws Exception {
     lock.lock();
     try {
@@ -1058,7 +1053,6 @@
     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 @@
     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 ba6da7a..ebe99da 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.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 @@
     }
   }
 
-  @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 7a943c4..09aeee5 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
@@ -31,8 +31,6 @@
 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;
 
 /**
@@ -78,22 +76,18 @@
     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);
   }
 
-  @VisibleForTesting
   public String getSyncReplicationStateNode(String peerId) {
     return ZNodePaths.joinZNode(getPeerNode(peerId), SYNC_REPLICATION_STATE_ZNODE);
   }
 
-  @VisibleForTesting
   public String getNewSyncReplicationStateNode(String peerId) {
     return ZNodePaths.joinZNode(getPeerNode(peerId), NEW_SYNC_REPLICATION_STATE_ZNODE);
   }
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 6f1f5a3..5c480ba 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.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 @@
    */
   private final String hfileRefsZNode;
 
-  @VisibleForTesting
   final String regionsZNode;
 
   public ZKReplicationQueueStorage(ZKWatcher zookeeper, Configuration conf) {
@@ -158,7 +156,6 @@
    * @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 @@
    * Return the {lastPushedSequenceId, ZNodeDataVersion} pair. if ZNodeDataVersion is -1, it means
    * that the ZNode does not exist.
    */
-  @VisibleForTesting
   protected Pair<Long, Integer> getLastSequenceIdWithVersion(String encodedRegionName,
       String peerId) throws KeeperException {
     Stat stat = new Stat();
@@ -503,7 +499,6 @@
   }
 
   // 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 @@
   }
 
   // will be overridden in UTs
-  @VisibleForTesting
   protected int getHFileRefsZNodeCversion() throws ReplicationException {
     Stat stat = new Stat();
     try {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ExecutorStatusChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ExecutorStatusChore.java
index da03eba..4b4aef3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ExecutorStatusChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ExecutorStatusChore.java
@@ -29,7 +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 Class ExecutorStatusChore for collect Executor status info periodically
@@ -77,7 +76,6 @@
     }
   }
 
-  @VisibleForTesting
   public Pair<Long, Long> getExecutorStatus(String poolName) {
     MutableGaugeLong running = metricsRegistry.getGauge(poolName + "_running", 0L);
     MutableGaugeLong queued = metricsRegistry.getGauge(poolName + "_queued", 0L);
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 e49fe8c..e27574a 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
@@ -35,8 +35,6 @@
 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;
-
 /**
  * Lock for HBase Entity either a Table, a Namespace, or Regions.
  *
@@ -145,12 +143,12 @@
     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 4229a7a..24f2835 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.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 @@
     return new EntityLock(conf, stub, lockRequest, abort);
   }
 
-  @VisibleForTesting
+  @InterfaceAudience.Private
   public static LockRequest buildLockRequest(final LockType type,
       final String namespace, final TableName tableName, final List<RegionInfo> regionInfos,
       final String description, final long nonceGroup, final long nonce) {
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 33d8f2c..9508321 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.Task;
 import org.apache.yetus.audience.InterfaceAudience;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 /**
  * Coordination for SplitLogManager. It creates and works with tasks for split log operations<BR>
  * Manager prepares task by calling {@link #prepareTask} and submit it by
@@ -147,6 +145,5 @@
    * 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 a9fae46..5452578 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.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 @@
    * 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 8fc351f..dee94be 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.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 @@
   /**
    * 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 7e0f2ae..4425076 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.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 @@
     }
   }
 
-  @VisibleForTesting
   public void load(Class<? extends C> implClass, int priority, Configuration conf)
       throws IOException {
     E env = checkAndLoadInstance(implClass, priority, conf);
@@ -325,7 +323,6 @@
     return null;
   }
 
-  @VisibleForTesting
   public <T extends C> T findCoprocessor(Class<T> cls) {
     for (E env: coprocEnvironments) {
       if (cls.isAssignableFrom(env.getInstance().getClass())) {
@@ -360,7 +357,6 @@
    * @param className the class name
    * @return the coprocessor, or null if not found
    */
-  @VisibleForTesting
   public E findCoprocessorEnvironment(String className) {
     for (E env: coprocEnvironments) {
       if (env.getInstance().getClass().getName().equals(className) ||
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetricsCoprocessor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetricsCoprocessor.java
index 8c1f9e3..a77a0fe 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetricsCoprocessor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetricsCoprocessor.java
@@ -25,8 +25,6 @@
 import org.apache.hadoop.hbase.metrics.MetricRegistry;
 import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 /**
  * Utility class for tracking metrics for various types of coprocessors. Each coprocessor instance
  * creates its own MetricRegistry which is exported as an individual MetricSource. MetricRegistries
@@ -75,7 +73,6 @@
         .toString();
   }
 
-  @VisibleForTesting
   static MetricRegistryInfo createRegistryInfoForMasterCoprocessor(String clazz) {
     return new MetricRegistryInfo(
         suffix(MASTER_COPROC_METRICS_NAME, clazz),
@@ -88,7 +85,6 @@
     return MetricRegistries.global().create(createRegistryInfoForMasterCoprocessor(clazz));
   }
 
-  @VisibleForTesting
   static MetricRegistryInfo createRegistryInfoForRSCoprocessor(String clazz) {
     return new MetricRegistryInfo(
         suffix(RS_COPROC_METRICS_NAME, clazz),
@@ -101,7 +97,6 @@
     return MetricRegistries.global().create(createRegistryInfoForRSCoprocessor(clazz));
   }
 
-  @VisibleForTesting
   public static MetricRegistryInfo createRegistryInfoForRegionCoprocessor(String clazz) {
     return new MetricRegistryInfo(
         suffix(REGION_COPROC_METRICS_NAME, clazz),
@@ -114,7 +109,6 @@
     return MetricRegistries.global().create(createRegistryInfoForRegionCoprocessor(clazz));
   }
 
-  @VisibleForTesting
   public static MetricRegistryInfo createRegistryInfoForWALCoprocessor(String clazz) {
     return new MetricRegistryInfo(
         suffix(WAL_COPROC_METRICS_NAME, clazz),
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/ObserverContextImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/ObserverContextImpl.java
index fdeca13..9a23ffa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/ObserverContextImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/ObserverContextImpl.java
@@ -23,8 +23,6 @@
 import org.apache.hadoop.hbase.security.User;
 import org.apache.yetus.audience.InterfaceAudience;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 /**
  * This is the only implementation of {@link ObserverContext}, which serves as the interface for
  * third-party Coprocessor developers.
@@ -98,7 +96,6 @@
    * @return An instance of <code>ObserverContext</code> with the environment set
    */
   @Deprecated
-  @VisibleForTesting
   // TODO: Remove this method, ObserverContext should not depend on RpcServer
   public static <E extends CoprocessorEnvironment> ObserverContext<E> createAndPrepare(E env) {
     ObserverContextImpl<E> ctx = new ObserverContextImpl<>(RpcServer.getRequestUser().orElse(null));
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 c12c30a..8d2d7e2 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.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 @@
    * 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) {
     Executor hbes = this.executorMap.compute(name, (key, value) -> {
       if (value != null) {
@@ -126,7 +123,6 @@
     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 a34e3a3..16f0934 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.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.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 @@
     datanodeDataTransferPort= getDataNodePort();
   }
 
-  @VisibleForTesting
   public int getDataNodePort() {
     HdfsConfiguration.init();
 
@@ -282,7 +282,6 @@
     }
   }
 
-  @VisibleForTesting
   public synchronized Set<RegionInfo> getRegionsOfFavoredNode(ServerName serverName) {
     Set<RegionInfo> 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 c1f9a7d..d83a9d9 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
@@ -28,14 +28,11 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.fs.HFileSystem;
-import org.apache.hadoop.hdfs.DFSInputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 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 +170,11 @@
   }
 
   /** 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 3634ccb..72da73e 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.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 @@
     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 efc83af..dcbb715 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.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.
  */
@@ -283,7 +281,6 @@
    * @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;
   }
@@ -329,7 +326,6 @@
    * @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 5544ece..1fb8713 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.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
@@ -397,7 +394,6 @@
     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 cae284a..7ab4edb 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 @@
     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 46cec4a..52b6359 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.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 @@
 
     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 f4fdb9b..a02ad7d 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.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;
 
 /**
@@ -311,7 +311,6 @@
    * @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,
@@ -524,7 +523,6 @@
     return this.allocator;
   }
 
-  @VisibleForTesting
   private void sanityCheckAssertion(long valueFromBuf, long valueFromField,
       String fieldName) throws IOException {
     if (valueFromBuf != valueFromField) {
@@ -533,7 +531,6 @@
     }
   }
 
-  @VisibleForTesting
   private void sanityCheckAssertion(BlockType valueFromBuf, BlockType valueFromField)
       throws IOException {
     if (valueFromBuf != valueFromField) {
@@ -550,7 +547,6 @@
    * 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();
@@ -839,7 +835,6 @@
     /**
      * @param dataBlockEncoder data block encoding algorithm to use
      */
-    @VisibleForTesting
     public Writer(HFileDataBlockEncoder dataBlockEncoder, HFileContext fileContext) {
       this(dataBlockEncoder, fileContext, ByteBuffAllocator.HEAP);
     }
@@ -1402,7 +1397,6 @@
     private long fileSize;
 
     /** The size of the header */
-    @VisibleForTesting
     protected final int hdrSize;
 
     /** The filesystem used to access data */
@@ -1693,7 +1687,6 @@
      * @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 {
@@ -1851,7 +1844,6 @@
   }
 
   /** An additional sanity-check in case no compression or encryption is being used. */
-  @VisibleForTesting
   void sanityCheckUncompressed() throws IOException {
     if (onDiskSizeWithoutHeader != uncompressedSizeWithoutHeader +
         totalChecksumBytes()) {
@@ -1973,7 +1965,6 @@
     return DataBlockEncoding.NONE;
   }
 
-  @VisibleForTesting
   byte getChecksumType() {
     return this.fileContext.getChecksumType().getCode();
   }
@@ -1983,7 +1974,6 @@
   }
 
   /** @return the size of data on disk + header. Excludes checksum. */
-  @VisibleForTesting
   int getOnDiskDataSizeWithHeader() {
     return this.onDiskDataSizeWithHeader;
   }
@@ -2022,7 +2012,6 @@
   /**
    * Return the appropriate DUMMY_HEADER for the minor version
    */
-  @VisibleForTesting
   // TODO: Why is this in here?
   byte[] getDummyHeaderForVersion() {
     return getDummyHeaderForVersion(this.fileContext.isUseHBaseChecksum());
@@ -2048,7 +2037,6 @@
    * 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 2060b20..1ed1bb5 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.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 @@
     return this.hfileContext.getCellComparator();
   }
 
-  @VisibleForTesting
   public Compression.Algorithm getCompressionAlgorithm() {
     return trailer.getCompressionCodec();
   }
@@ -1627,7 +1625,6 @@
    * not completed, true otherwise
    */
   @Override
-  @VisibleForTesting
   public boolean prefetchComplete() {
     return PrefetchExecutor.isCompleted(path);
   }
@@ -1645,7 +1642,6 @@
    * @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 60aa65d..b1a9848 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.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.
  */
@@ -784,7 +782,6 @@
     }
   }
 
-  @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 b2016ab..bc0df43 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.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;
@@ -636,12 +635,10 @@
     }
   }
 
-  @VisibleForTesting
   boolean isEvictionInProgress() {
     return evictionInProgress;
   }
 
-  @VisibleForTesting
   long getOverhead() {
     return overhead;
   }
@@ -1173,7 +1170,6 @@
   }
 
   /** Clears the cache. Used in tests. */
-  @VisibleForTesting
   public void clearCache() {
     this.map.clear();
     this.elements.set(0);
@@ -1184,7 +1180,6 @@
    *
    * @return the set of cached file names
    */
-  @VisibleForTesting
   SortedSet<String> getCachedFileNamesForTest() {
     SortedSet<String> fileNames = new TreeSet<>();
     for (BlockCacheKey cacheKey : map.keySet()) {
@@ -1193,7 +1188,6 @@
     return fileNames;
   }
 
-  @VisibleForTesting
   public Map<DataBlockEncoding, Integer> getEncodingCountsForTest() {
     Map<DataBlockEncoding, Integer> counts = new EnumMap<>(DataBlockEncoding.class);
     for (LruCachedBlock block : map.values()) {
@@ -1204,7 +1198,6 @@
     return counts;
   }
 
-  @VisibleForTesting
   Map<BlockCacheKey, LruCachedBlock> 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 c58d5b8..1f903cf 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.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 @@
     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 a90c5a3..a0dc30c 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.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 @@
 
   private BlockCache victimCache;
 
-  @VisibleForTesting
   final Cache<BlockCacheKey, Cacheable> 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 a205d27..a84d812 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.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;
@@ -79,7 +78,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 
@@ -119,7 +117,6 @@
   private static final boolean STRONG_REF_DEFAULT = false;
 
   /** 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;
@@ -141,10 +138,8 @@
   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<BlockCacheKey, BucketEntry> backingMap;
 
   /**
@@ -161,9 +156,7 @@
    * 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<BlockingQueue<RAMQueueEntry>> writerQueues = new ArrayList<>();
-  @VisibleForTesting
   transient final WriterThread[] writerThreads;
 
   /** Volatile boolean to track if free space is in process or not */
@@ -185,7 +178,6 @@
    * 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();
@@ -209,7 +201,6 @@
    * The purpose of this is to avoid freeing the block which is being read.
    * <p>
    */
-  @VisibleForTesting
   transient final IdReadWriteLock<Long> offsetLock;
 
   private final NavigableSet<BlockCacheKey> blocksByHFile = new ConcurrentSkipListSet<>((a, b) -> {
@@ -352,14 +343,12 @@
    * 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;
   }
@@ -556,7 +545,6 @@
     return null;
   }
 
-  @VisibleForTesting
   void blockEvicted(BlockCacheKey cacheKey, BucketEntry bucketEntry, boolean decrementBlockNumber) {
     bucketAllocator.freeBlock(bucketEntry.offset());
     realCacheSize.add(-1 * bucketEntry.getLength());
@@ -681,7 +669,6 @@
     return (long) Math.floor(bucketAllocator.getTotalSize() * acceptableFactor);
   }
 
-  @VisibleForTesting
   long getPartitionSize(float partitionFactor) {
     return (long) Math.floor(bucketAllocator.getTotalSize() * partitionFactor * minFactor);
   }
@@ -876,7 +863,6 @@
   }
 
   // This handles flushing the RAM cache to IOEngine.
-  @VisibleForTesting
   class WriterThread extends Thread {
     private final BlockingQueue<RAMQueueEntry> inputQueue;
     private volatile boolean writerEnabled = true;
@@ -887,7 +873,6 @@
     }
 
     // Used for test
-    @VisibleForTesting
     void disableWriter() {
       this.writerEnabled = false;
     }
@@ -947,7 +932,6 @@
      *   interference expected.
      * @throws InterruptedException
      */
-    @VisibleForTesting
     void doDrain(final List<RAMQueueEntry> entries) throws InterruptedException {
       if (entries.isEmpty()) {
         return;
@@ -1055,7 +1039,6 @@
    * @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<RAMQueueEntry> getRAMQueueEntries(BlockingQueue<RAMQueueEntry> q,
       List<RAMQueueEntry> receptacle) throws InterruptedException {
     // Clear sets all entries to null and sets size to 0. We retain allocations. Presume it
@@ -1349,7 +1332,6 @@
   /**
    * Block Entry stored in the memory with key,data and so on
    */
-  @VisibleForTesting
   static class RAMQueueEntry {
     private final BlockCacheKey key;
     private final Cacheable data;
@@ -1531,7 +1513,6 @@
     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 2cdfc80..e4a2c0b 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.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.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 @@
     return be.wrapAsCacheable(dstBuff);
   }
 
-  @VisibleForTesting
   void closeFileChannels() {
     for (FileChannel fileChannel: fileChannels) {
       try {
@@ -283,12 +280,10 @@
     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 eab2a0e..f34cad5 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.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 @@
     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 bac19f1..855cf2f 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 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 @@
     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 7bae06f..ca8593e 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.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.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 @@
    * @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 99e0188..e06daac 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 @@
 
 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.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 @@
    * 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 2e7baae..2f75560 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.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 @@
     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 7adbeb0..7d29ed6 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
@@ -218,7 +218,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
 import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
@@ -267,7 +266,7 @@
   // 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;
@@ -621,7 +620,7 @@
     }
   }
 
-  @VisibleForTesting
+  @InterfaceAudience.Private
   public MasterRpcServices getMasterRpcServices() {
     return (MasterRpcServices)rpcServices;
   }
@@ -751,7 +750,7 @@
   }
 
   // Will be overriden in test to inject customized AssignmentManager
-  @VisibleForTesting
+  @InterfaceAudience.Private
   protected AssignmentManager createAssignmentManager(MasterServices master) {
     return new AssignmentManager(master);
   }
@@ -1117,7 +1116,7 @@
    * @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);
   }
@@ -1188,7 +1187,7 @@
    * 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(
@@ -1221,7 +1220,7 @@
    * Will be overridden in tests.
    * </p>
    */
-  @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.
@@ -1235,7 +1234,7 @@
   }
 
   // Will be overridden in tests
-  @VisibleForTesting
+  @InterfaceAudience.Private
   protected void initClusterSchemaService() throws IOException, InterruptedException {
     this.clusterSchemaService = new ClusterSchemaServiceImpl(this);
     this.clusterSchemaService.startAsync();
@@ -1912,7 +1911,7 @@
   // 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 IOException {
     RegionState regionState = assignmentManager.getRegionStates().
       getRegionState(Bytes.toString(encodedRegionName));
@@ -2740,7 +2739,7 @@
     }
   }
 
-  @VisibleForTesting
+  @InterfaceAudience.Private
   protected void checkServiceStarted() throws ServerNotRunningYetException {
     if (!serviceStarted) {
       throw new ServerNotRunningYetException("Server is not running yet");
@@ -2795,7 +2794,7 @@
     return maintenanceMode;
   }
 
-  @VisibleForTesting
+  @InterfaceAudience.Private
   public void setInitialized(boolean isInitialized) {
     procedureExecutor.getEnvironment().setEventReady(initialized, isInitialized);
   }
@@ -3643,7 +3642,7 @@
   /**
    * 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 0bbfa4a..ca7eb90 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
@@ -43,8 +43,6 @@
 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
@@ -293,7 +291,6 @@
    * 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 c5f0f3c..f24ecd4 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
@@ -56,7 +56,7 @@
 import org.apache.hadoop.hbase.security.access.ZKPermissionWatcher;
 import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
 import org.apache.hbase.thirdparty.com.google.protobuf.Service;
 
 /**
@@ -140,7 +140,6 @@
   /**
    * @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 6001c8f..61b4327 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.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 @@
   /**
    * 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) {
@@ -124,7 +122,6 @@
     }
   }
 
-  @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 d1d9352..9e666c5 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
@@ -68,7 +68,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
 
@@ -276,7 +275,6 @@
     }
   }
 
-  @VisibleForTesting
   public void regionServerReport(ServerName sn,
     ServerMetrics sl) throws YouAreDeadException {
     checkIsDead(sn, "REPORT");
@@ -428,13 +426,11 @@
    * 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);
   }
 
-  @VisibleForTesting
   public ConcurrentNavigableMap<byte[], Long> getFlushedSequenceIdByRegion() {
     return flushedSequenceIdByRegion;
   }
@@ -570,7 +566,7 @@
    *         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);
 
@@ -629,7 +625,6 @@
    * 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);
@@ -996,7 +991,6 @@
     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 3e0c746..465a593 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.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 @@
   private long unassignedTimeout;
   private long lastTaskCreateTime = Long.MAX_VALUE;
 
-  @VisibleForTesting
   final ConcurrentMap<String, Task> tasks = new ConcurrentHashMap<>();
   private TimeoutMonitor timeoutMonitor;
 
@@ -165,7 +163,6 @@
    * {@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<Path> logDirs,
       final PathFilter filter)
       throws IOException {
@@ -375,7 +372,6 @@
     }
   }
 
-  @VisibleForTesting
   ConcurrentMap<String, Task> 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 9ff84dc..48c19c2 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.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 @@
     return !fs.exists(new Path(rootDir, walPath));
   }
 
-  @VisibleForTesting
   List<Procedure> createSplitWALProcedures(List<FileStatus> 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 25c94ba..fe304ce 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.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 @@
     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 355dfde..4d0e165 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
@@ -86,8 +86,6 @@
 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;
@@ -181,7 +179,6 @@
     this(master, new RegionStateStore(master));
   }
 
-  @VisibleForTesting
   AssignmentManager(final MasterServices master, final RegionStateStore stateStore) {
     this.master = master;
     this.regionStateStore = stateStore;
@@ -513,7 +510,6 @@
    * 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";
@@ -783,7 +779,6 @@
     return createRoundRobinAssignProcedures(hris, null);
   }
 
-  @VisibleForTesting
   static int compare(TransitRegionStateProcedure left, TransitRegionStateProcedure right) {
     if (left.getRegion().isMetaRegion()) {
       if (right.getRegion().isMetaRegion()) {
@@ -1368,7 +1363,6 @@
     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);
@@ -2270,7 +2264,6 @@
         .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 5e06a44..80a61da 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
@@ -59,7 +59,7 @@
 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;
@@ -752,7 +752,6 @@
   /**
    * @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 b7fcdab..d04dbef 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.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 @@
     }
   }
 
-  @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/RegionStateStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
index c8b0e35..4da9493 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
@@ -71,7 +71,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -464,7 +463,6 @@
       qualifiers.stream().map(Bytes::toStringBinary).collect(Collectors.joining(", ")));
   }
 
-  @VisibleForTesting
   static Put addMergeRegions(Put put, Collection<RegionInfo> mergeRegions) throws IOException {
     int limit = 10000; // Arbitrary limit. No room in our formatted 'task0000' below for more.
     int max = mergeRegions.size();
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 0637800..147a112 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.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
@@ -115,7 +113,6 @@
     serverMap.clear();
   }
 
-  @VisibleForTesting
   public boolean isRegionInRegionStates(final RegionInfo hri) {
     return (regionsMap.containsKey(hri.getRegionName()) || regionInTransition.containsKey(hri)
         || regionOffline.containsKey(hri));
@@ -124,7 +121,6 @@
   // ==========================================================================
   //  RegionStateNode helpers
   // ==========================================================================
-  @VisibleForTesting
   RegionStateNode createRegionStateNode(RegionInfo regionInfo) {
     synchronized (regionsMapLock) {
       RegionStateNode node = regionsMap.computeIfAbsent(regionInfo.getRegionName(),
@@ -764,7 +760,6 @@
   /**
    * @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 2e76771..bc46e19 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.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 @@
     this.regionInfo = regionInfo;
   }
 
-  @VisibleForTesting
   public RegionInfo getRegionInfo() {
     return regionInfo;
   }
 
-  @VisibleForTesting
   public void setRegionInfo(final RegionInfo regionInfo) {
     this.regionInfo = regionInfo;
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
index 0eb7667..59cc33f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
@@ -74,8 +74,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.util.concurrent.ThreadFactoryBuilder;
+
 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;
@@ -156,12 +157,10 @@
       daughterTwoRI);
   }
 
-  @VisibleForTesting
   public RegionInfo getDaughterOneRI() {
     return daughterOneRI;
   }
 
-  @VisibleForTesting
   public RegionInfo getDaughterTwoRI() {
     return daughterTwoRI;
   }
@@ -484,7 +483,6 @@
    * Prepare to Split region.
    * @param env MasterProcedureEnv
    */
-  @VisibleForTesting
   public boolean prepareSplitRegion(final MasterProcedureEnv env) throws IOException {
     // Fail if we are taking snapshot for the given table
     if (env.getMasterServices().getSnapshotManager()
@@ -600,7 +598,6 @@
   /**
    * Create daughter regions
    */
-  @VisibleForTesting
   public void createDaughterRegions(final MasterProcedureEnv env) throws IOException {
     final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
     final Path tabledir = CommonFSUtils.getTableDir(mfs.getRootDir(), getTableName());
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/TransitRegionStateProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/TransitRegionStateProcedure.java
index 63bb345..56e3215 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/TransitRegionStateProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/TransitRegionStateProcedure.java
@@ -41,8 +41,6 @@
 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.RegionStateTransitionState;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionStateTransitionStateData;
@@ -143,7 +141,6 @@
     }
   }
 
-  @VisibleForTesting
   protected TransitRegionStateProcedure(MasterProcedureEnv env, RegionInfo hri,
       ServerName assignCandidate, boolean forceNewPlan, TransitionType type) {
     super(env, hri);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
index 5ecedb3..91215c7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
@@ -37,7 +37,6 @@
 import java.util.TreeMap;
 import java.util.function.Predicate;
 import java.util.stream.Collectors;
-
 import org.apache.commons.lang3.NotImplementedException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ClusterMetrics;
@@ -58,14 +57,14 @@
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.Action.Type;
 import org.apache.hadoop.hbase.namequeues.NamedQueueRecorder;
 import org.apache.hadoop.hbase.net.Address;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.base.Joiner;
 import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap;
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * The base class for load balancers. It provides the the functions used to by
@@ -1000,12 +999,10 @@
       }
     }
 
-    @VisibleForTesting
     protected void setNumRegions(int numRegions) {
       this.numRegions = numRegions;
     }
 
-    @VisibleForTesting
     protected void setNumMovedRegions(int numMovedRegions) {
       this.numMovedRegions = numMovedRegions;
     }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
index eb1c2bc..3f249b8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
@@ -29,7 +29,6 @@
 import java.util.Objects;
 import java.util.Random;
 import java.util.stream.Collectors;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
@@ -56,10 +55,8 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
-
 /**
  * <p>This is a best effort load balancer. Given a Cost function F(C) =&gt; x It will
  * randomly try and mutate the cluster to Cprime. If F(Cprime) &lt; F(C) then the
@@ -365,7 +362,6 @@
     return !balanced;
   }
 
-  @VisibleForTesting
   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 4331d49..9416e5a 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.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.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 @@
     }
   }
 
-  @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 6926f12..ff28857 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.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 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 @@
     }
   }
 
-  @VisibleForTesting
   public List<Thread> 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 5fa115c..f65713e 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.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.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 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<CleanerContext> pendingDelete;
@@ -138,12 +135,10 @@
     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 6123f6b..45b6a74 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
@@ -60,8 +60,6 @@
 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 <code>hbase:meta</code> catalog table on a period.
  * Makes a lastReport on state of hbase:meta. Looks for unused regions to garbage collect. Scan of
@@ -218,7 +216,6 @@
    * @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.
@@ -304,7 +301,6 @@
     }
   }
 
-  @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 f9dc1cc..4a5aa0a 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.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 @@
    * @param maxMergeCount Maximum regions to merge at a time (avoid merging
    *   100k regions in one go!)
    */
-  @VisibleForTesting
   static List<SortedSet<RegionInfo>> calculateMerges(int maxMergeCount,
       List<Pair<RegionInfo, RegionInfo>> overlaps) {
     if (overlaps.isEmpty()) {
@@ -333,7 +330,6 @@
    * @return Either <code>a</code> or <code>b</code>, 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 @@
    * @return True if an overlap found between passed in <code>ri</code> and
    *   the <code>pair</code>. Does NOT check the pairs themselves overlap.
    */
-  @VisibleForTesting
   static boolean isOverlap(RegionInfo ri, Pair<RegionInfo, RegionInfo> 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 26b838d..aaf5152 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.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 @@
       return "MasterLock: proc = " + proc.toString();
     }
 
-    @VisibleForTesting
     LockProcedure getProc() {
       return proc;
     }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java
index e027a73..bd6d44c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java
@@ -29,8 +29,6 @@
 import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 /**
  * Base class for all the Namespace procedures that want to use a StateMachineProcedure. It provide
  * some basic helpers like basic locking and basic toStringClassDetails().
@@ -114,7 +112,6 @@
     createDirectory(env.getMasterServices().getMasterFileSystem(), nsDescriptor);
   }
 
-  @VisibleForTesting
   public static void createDirectory(MasterFileSystem mfs, NamespaceDescriptor nsDescriptor)
     throws IOException {
     mfs.getFileSystem()
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 9ffe3c6..2313e70 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
@@ -47,7 +47,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -421,7 +420,6 @@
     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 c470c42..456660d 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
@@ -42,8 +42,6 @@
 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
@@ -659,7 +657,6 @@
    * @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 {
@@ -1015,7 +1012,6 @@
   /**
    * 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 1942ed6..2f990cb 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
@@ -41,9 +41,10 @@
 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.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest;
@@ -377,7 +378,6 @@
     }
 
     // will be overridden in test.
-    @VisibleForTesting
     protected ExecuteProceduresResponse sendRequest(final ServerName serverName,
         final ExecuteProceduresRequest request) throws IOException {
       return FutureUtils.get(getRsAdmin().executeProcedures(request));
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 4ae408f..f882231 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.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,8 +164,7 @@
     return walPath;
   }
 
-  @VisibleForTesting
-  public ServerName getWorker(){
+  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 290e0c1..26eceb9 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.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.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 @@
     }
   }
 
-  @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 688a549..c2188b4 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.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 @@
 
   private final WALFactory walFactory;
 
-  @VisibleForTesting
   final HRegion region;
 
-  @VisibleForTesting
   final MasterRegionFlusherAndCompactor flusherAndCompactor;
 
   private MasterRegionWALRoller walRoller;
@@ -141,17 +138,14 @@
     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/AbstractPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java
index fc254a3..8985d2f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java
@@ -38,8 +38,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 /**
  * The base class for all replication peer related procedure.
  */
@@ -93,7 +91,6 @@
   }
 
   // will be override in test to simulate error
-  @VisibleForTesting
   protected void enablePeer(MasterProcedureEnv env) throws ReplicationException {
     env.getReplicationPeerManager().enablePeer(peerId);
   }
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 8dd329f..7401c4b 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
@@ -34,8 +34,6 @@
 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;
 
 /**
@@ -133,7 +131,6 @@
   }
 
   // will be override in test to simulate error
-  @VisibleForTesting
   protected void reopenRegions(MasterProcedureEnv env) throws IOException {
     ReplicationPeerConfig peerConfig = getNewPeerConfig();
     ReplicationPeerConfig oldPeerConfig = getOldPeerConfig();
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/SyncReplicationReplayWALManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/SyncReplicationReplayWALManager.java
index ae624b1..419e79b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/SyncReplicationReplayWALManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/SyncReplicationReplayWALManager.java
@@ -47,8 +47,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 /**
  * The manager for replaying remote wal.
  * <p/>
@@ -289,7 +287,6 @@
     return fs.getFileStatus(walPath).getLen() == 0;
   }
 
-  @VisibleForTesting
   public Path getRemoteWALDir() {
     return remoteWALDir;
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java
index 358fd5e..289e012 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java
@@ -36,8 +36,6 @@
 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.PeerSyncReplicationStateTransitionState;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.TransitPeerSyncReplicationStateStateData;
 
@@ -110,7 +108,6 @@
     return PeerSyncReplicationStateTransitionState.PRE_PEER_SYNC_REPLICATION_STATE_TRANSITION;
   }
 
-  @VisibleForTesting
   protected void preTransit(MasterProcedureEnv env) throws IOException {
     MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
     if (cpHost != null) {
@@ -144,14 +141,12 @@
     }
   }
 
-  @VisibleForTesting
   protected void reopenRegions(MasterProcedureEnv env) {
     addChildProcedure(
       env.getReplicationPeerManager().getPeerConfig(peerId).get().getTableCFsMap().keySet().stream()
         .map(ReopenTableRegionsProcedure::new).toArray(ReopenTableRegionsProcedure[]::new));
   }
 
-  @VisibleForTesting
   protected void createDirForRemoteWAL(MasterProcedureEnv env) throws IOException {
     MasterFileSystem mfs = env.getMasterFileSystem();
     Path remoteWALDir = new Path(mfs.getWALRootDir(), ReplicationUtils.REMOTE_WAL_DIR_NAME);
@@ -204,7 +199,6 @@
     addChildProcedure(new RecoverStandbyProcedure(peerId, serial));
   }
 
-  @VisibleForTesting
   protected void setPeerNewSyncReplicationState(MasterProcedureEnv env)
       throws ReplicationException {
     if (toState.equals(SyncReplicationState.STANDBY) ||
@@ -223,12 +217,10 @@
     env.getReplicationPeerManager().setPeerNewSyncReplicationState(peerId, toState);
   }
 
-  @VisibleForTesting
   protected void removeAllReplicationQueues(MasterProcedureEnv env) throws ReplicationException {
     env.getReplicationPeerManager().removeAllQueues(peerId);
   }
 
-  @VisibleForTesting
   protected void transitPeerSyncReplicationState(MasterProcedureEnv env)
       throws ReplicationException {
     env.getReplicationPeerManager().transitPeerSyncReplicationState(peerId, toState);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotFileCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotFileCache.java
index 039988a..fe46e4c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotFileCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotFileCache.java
@@ -41,7 +41,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
 /**
@@ -271,7 +270,6 @@
     this.snapshots.putAll(newSnapshots);
   }
 
-  @VisibleForTesting
   List<String> getSnapshotsInProgress() throws IOException {
     List<String> 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 8f57e6e..3010779 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.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 @@
    * @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/MobFileCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCleanerChore.java
index 82a08a9..13f43fc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCleanerChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCleanerChore.java
@@ -52,7 +52,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.collect.SetMultimap;
 
 /**
@@ -101,7 +100,6 @@
     }
   }
 
-  @VisibleForTesting
   public MobFileCleanerChore() {
     this.master = null;
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCompactionChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCompactionChore.java
index dd5d289..7fe2d0d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCompactionChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCompactionChore.java
@@ -41,9 +41,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
-
 /**
  * Periodic MOB compaction chore.
  * It runs MOB compaction on region servers in parallel, thus
@@ -74,7 +71,6 @@
 
   }
 
-  @VisibleForTesting
   public MobFileCompactionChore(Configuration conf, int batchSize) {
     this.regionBatchSize = batchSize;
   }
@@ -143,7 +139,6 @@
     }
   }
 
-  @VisibleForTesting
   public void performMajorCompactionInBatches(Admin admin, TableDescriptor htd,
       ColumnFamilyDescriptor hcd) throws IOException, InterruptedException {
 
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 9e58daa..720f25b 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
@@ -28,16 +28,14 @@
 import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
-
 import org.apache.hadoop.hbase.errorhandling.ForeignException;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
 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.common.collect.MapMaker;
 
 /**
@@ -71,7 +69,7 @@
    *
    * @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 36a919d..301b605 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.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 @@
 
   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 b245795..d10e6ea 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 @@
 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 @@
   }
 
   // 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 bbe53b2..b7cd26f 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
@@ -27,8 +27,6 @@
 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}.
  */
@@ -47,12 +45,10 @@
     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 edda4df..501ad8b 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 @@
     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 e67eda5..50f40af 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 @@
 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 @@
   }
 
   // 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 d80134a..ff4c519 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.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 @@
     }
   }
 
-  @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 1c97b20..bd1bff1 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
@@ -23,7 +23,6 @@
 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;
@@ -180,32 +179,26 @@
     return computeIfAbsent(quotasMap, key, QuotaState::new, this::triggerCacheRefresh);
   }
 
-  @VisibleForTesting
   void triggerCacheRefresh() {
     refreshChore.triggerNow();
   }
 
-  @VisibleForTesting
   long getLastUpdate() {
     return refreshChore.lastUpdate;
   }
 
-  @VisibleForTesting
   Map<String, QuotaState> getNamespaceQuotaCache() {
     return namespaceQuotaCache;
   }
 
-  @VisibleForTesting
   Map<String, QuotaState> getRegionServerQuotaCache() {
     return regionServerQuotaCache;
   }
 
-  @VisibleForTesting
   Map<TableName, QuotaState> getTableQuotaCache() {
     return tableQuotaCache;
   }
 
-  @VisibleForTesting
   Map<String, UserQuotaState> 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 425c240..0a8bfe1 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.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.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 @@
     }
   }
 
-  @VisibleForTesting
   QuotaSnapshotStore<TableName> getTableSnapshotStore() {
     return tableSnapshotStore;
   }
 
-  @VisibleForTesting
   QuotaSnapshotStore<String> 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 852d8a6..a29b90d 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 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 @@
   }
 
   // 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 0f96de5..8eee70e 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
@@ -33,7 +33,7 @@
 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 +83,6 @@
     }
   }
 
-  @VisibleForTesting
   protected boolean isRpcThrottleEnabled() {
     return rpcThrottleEnabled;
   }
@@ -108,7 +107,6 @@
     }
   }
 
-  @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 b9797bc..81e7e87 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.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 @@
     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 60cc92f..0d5f0b5 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;
@@ -28,12 +27,12 @@
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.ExtendedCell;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
 import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
 
 /**
  * An abstract class, which implements the behaviour shared by all concrete memstore instances.
@@ -377,12 +376,10 @@
     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 2dbc106..df62957 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 @@
 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.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 @@
     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 d74655d..e1cfd15 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 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 @@
     this.numUniqueKeys = numUniqueKeys;
   }
 
-  @VisibleForTesting
   CellSet(final NavigableMap<Cell, Cell> m) {
     this.delegatee = m;
     this.numUniqueKeys = UNKNOWN_NUM_UNIQUES;
   }
 
-  @VisibleForTesting
   NavigableMap<Cell, Cell> 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 136efee..1023890 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 @@
 
 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 @@
         + (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 d9f3271..5245ac2 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.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 @@
   private Map<Integer, Chunk> chunkIdMap = new ConcurrentHashMap<Integer, Chunk>();
 
   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 @@
             initialCountPercentage, heapMemoryManager);
   }
 
-  @VisibleForTesting
   private void initializePools(int chunkSize, long globalMemStoreSize,
                                float poolSizePercentage, float indexChunkSizePercentage,
                                float initialCountPercentage,
@@ -122,7 +116,6 @@
    */
   @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 @@
     return instance;
   }
 
-  @VisibleForTesting
   public static ChunkCreator getInstance() {
     return instance;
   }
@@ -280,8 +272,7 @@
     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 @@
     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 @@
     }
   }
 
-  @VisibleForTesting
   static void clearDisableFlag() {
     chunkPoolDisabled = false;
   }
@@ -507,12 +495,10 @@
     return memStoreChunkPool;
   }
 
-  @VisibleForTesting
   int getMaxCount() {
     return getMaxCount(ChunkType.DATA_CHUNK);
   }
 
-  @VisibleForTesting
   int getMaxCount(ChunkType chunkType) {
     switch (chunkType) {
       case INDEX_CHUNK:
@@ -533,12 +519,10 @@
     return 0;
   }
 
-  @VisibleForTesting
   int getPoolSize() {
     return getPoolSize(ChunkType.DATA_CHUNK);
   }
 
-  @VisibleForTesting
   int getPoolSize(ChunkType chunkType) {
     switch (chunkType) {
       case INDEX_CHUNK:
@@ -558,7 +542,6 @@
     return 0;
   }
 
-  @VisibleForTesting
   boolean isChunkInPool(int chunkId) {
     Chunk c = getChunk(chunkId);
     if (c==null) {
@@ -577,7 +560,6 @@
   /*
    * 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 7516c54..441b18b 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 @@
 
 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.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 @@
     // 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 2f24135..2be04c2 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.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 @@
   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 @@
    * @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 @@
    * cleanup. Use this method to set no-executor before you call run.
    * @return The old setting for <code>useExecutor</code>
    */
-  @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 7629273..abe9cf8 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.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 @@
   // 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 @@
         (this.compactor == null? "NULL": this.compactor.toString()));
   }
 
-  @VisibleForTesting
   protected MemStoreCompactor createMemStoreCompactor(MemoryCompactionPolicy compactionPolicy)
       throws IllegalArgumentIOException {
     return new MemStoreCompactor(this, compactionPolicy);
@@ -334,7 +331,6 @@
   }
 
   // the getSegments() method is used for tests only
-  @VisibleForTesting
   @Override
   protected List<Segment> getSegments() {
     List<? extends Segment> pipelineList = pipeline.getSegments();
@@ -367,7 +363,6 @@
   }
 
   // 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 @@
     return list;
   }
 
-  @VisibleForTesting
   protected List<KeyValueScanner> createList(int capacity) {
     return new ArrayList<>(capacity);
   }
@@ -451,7 +445,6 @@
   // 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 @@
     return getRegionServices().getInMemoryCompactionPool();
   }
 
-  @VisibleForTesting
   protected boolean shouldFlushInMemory(MutableSegment currActive, Cell cellToAdd,
       MemStoreSizing memstoreSizing) {
     long cellSize = MutableSegment.getCellLength(cellToAdd);
@@ -596,7 +588,6 @@
     }
   }
 
-  @VisibleForTesting
   boolean isMemStoreFlushingInMemory() {
     return inMemoryCompactionInProgress.get();
   }
@@ -619,7 +610,6 @@
     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 19647fa..53ef82d 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.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 @@
     }
   }
 
-  @VisibleForTesting
   @Override
   public List<Segment> 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 5ffd960..1f22dc4 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 @@
     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 c30f586..fc3c623 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
@@ -189,7 +189,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
@@ -319,7 +318,6 @@
 
   // Track data size in all memstores
   private final MemStoreSizing memStoreSizing = new ThreadSafeMemStoreSizing();
-  @VisibleForTesting
   RegionServicesForStores regionServicesForStores;
 
   // Debug possible data loss due to WAL off
@@ -581,7 +579,6 @@
   }
 
   /** A result object from prepare flush cache stage */
-  @VisibleForTesting
   static class PrepareFlushResult {
     final FlushResultImpl result; // indicating a failure result from prepare
     final TreeMap<byte[], StoreFlushContext> storeFlushCtxs;
@@ -746,7 +743,6 @@
    * @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) {
@@ -951,7 +947,6 @@
    * @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
@@ -1230,7 +1225,6 @@
     return allStoreFiles;
   }
 
-  @VisibleForTesting
   protected void writeRegionOpenMarker(WAL wal, long openSeqId) throws IOException {
     Map<byte[], List<Path>> storeFiles = getStoreFiles();
     RegionEventDescriptor regionOpenDesc = ProtobufUtil.toRegionEventDescriptor(
@@ -1514,7 +1508,6 @@
     }
   }
 
-  @VisibleForTesting
   public MultiVersionConcurrencyControl getMVCC() {
     return mvcc;
   }
@@ -1637,7 +1630,6 @@
   /**
    * Exposed for some very specific unit tests.
    */
-  @VisibleForTesting
   public void setClosing(boolean closing) {
     this.closing.set(closing);
   }
@@ -1647,7 +1639,6 @@
    * 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;
@@ -2045,7 +2036,6 @@
     return this.htableDescriptor;
   }
 
-  @VisibleForTesting
   public void setTableDescriptor(TableDescriptor desc) {
     htableDescriptor = desc;
   }
@@ -2062,7 +2052,6 @@
   /**
    * Only used for unit test which doesn't start region server.
    */
-  @VisibleForTesting
   public void setBlockCache(BlockCache blockCache) {
     this.blockCache = blockCache;
   }
@@ -2074,7 +2063,6 @@
   /**
    * Only used for unit test which doesn't start region server.
    */
-  @VisibleForTesting
   public void setMobFileCache(MobFileCache mobFileCache) {
     this.mobFileCache = mobFileCache;
   }
@@ -2082,7 +2070,6 @@
   /**
    * @return split policy for this region.
    */
-  @VisibleForTesting
   RegionSplitPolicy getSplitPolicy() {
     return this.splitPolicy;
   }
@@ -2126,7 +2113,6 @@
    * @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(),
@@ -2235,7 +2221,6 @@
    * <p>
    * It is used by utilities and testing
    */
-  @VisibleForTesting
   public void compactStores() throws IOException {
     for (HStore s : stores.values()) {
       Optional<CompactionContext> compaction = s.requestCompaction();
@@ -2250,7 +2235,6 @@
    * <p>
    * It is used by utilities and testing
    */
-  @VisibleForTesting
   void compactStore(byte[] family, ThroughputController throughputController) throws IOException {
     HStore s = getStore(family);
     Optional<CompactionContext> compaction = s.requestCompaction();
@@ -3131,7 +3115,6 @@
    * @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);
@@ -5306,7 +5289,6 @@
    * @return the sequence id of the last edit added to this region out of the
    * recovered edits log or <code>minSeqId</code> if nothing added from editlogs.
    */
-  @VisibleForTesting
   long replayRecoveredEditsIfAny(Map<byte[], Long> maxSeqIdInStores,
     final CancelableProgressable reporter, final MonitoredTask status) throws IOException {
     long minSeqIdForTheRegion = -1;
@@ -5772,7 +5754,6 @@
    * 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();
 
@@ -5883,7 +5864,6 @@
     return null;
   }
 
-  @VisibleForTesting
   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
     justification="Intentional; post memstore flush")
   void replayWALFlushCommitMarker(FlushDescriptor flush) throws IOException {
@@ -6171,7 +6151,6 @@
     }
   }
 
-  @VisibleForTesting
   PrepareFlushResult getPrepareFlushResult() {
     return prepareFlushResult;
   }
@@ -6535,7 +6514,6 @@
    * @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);
   }
@@ -6738,7 +6716,6 @@
     }
   }
 
-  @VisibleForTesting
   public int getReadLockCount() {
     return lock.getReadLockCount();
   }
@@ -6747,7 +6724,6 @@
     return lockedRows;
   }
 
-  @VisibleForTesting
   class RowLockContext {
     private final HashedBytes row;
     final ReadWriteLock readWriteLock = new ReentrantReadWriteLock(true);
@@ -6824,7 +6800,6 @@
       return lock;
     }
 
-    @VisibleForTesting
     public RowLockContext getContext() {
       return context;
     }
@@ -8114,7 +8089,6 @@
     return r.openHRegion(reporter);
   }
 
-  @VisibleForTesting
   public NavigableMap<byte[], Integer> getReplicationScope() {
     return this.replicationScope;
   }
@@ -8890,7 +8864,6 @@
   }
 
   /** @param coprocessorHost the new coprocessor host */
-  @VisibleForTesting
   public void setCoprocessorHost(final RegionCoprocessorHost coprocessorHost) {
     this.coprocessorHost = coprocessorHost;
   }
@@ -9190,7 +9163,6 @@
     flushesQueued.increment();
   }
 
-  @VisibleForTesting
   public long getReadPoint() {
     return getReadPoint(IsolationLevel.READ_COMMITTED);
   }
@@ -9390,12 +9362,10 @@
     }
   }
 
-  @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 d5ef30e..73234f1 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.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 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 c4261dc..1c2f68c 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
@@ -175,7 +175,6 @@
 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;
@@ -192,7 +191,6 @@
 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;
@@ -253,7 +251,7 @@
   /**
    * 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;
 
@@ -545,7 +543,7 @@
    */
   protected final ConfigurationManager configurationManager;
 
-  @VisibleForTesting
+  @InterfaceAudience.Private
   CompactedHFilesDischarger compactedFileDischarger;
 
   private volatile ThroughputController flushThroughputController;
@@ -1244,7 +1242,7 @@
     return writeCount;
   }
 
-  @VisibleForTesting
+  @InterfaceAudience.Private
   protected void tryRegionServerReport(long reportStartTime, long reportEndTime)
       throws IOException {
     RegionServerStatusService.BlockingInterface rss = rssStub;
@@ -2521,7 +2519,7 @@
     return rpcServices.rpcServer;
   }
 
-  @VisibleForTesting
+  @InterfaceAudience.Private
   public RSRpcServices getRSRpcServices() {
     return rpcServices;
   }
@@ -2611,7 +2609,7 @@
    * 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");
@@ -2720,7 +2718,7 @@
    * @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();
@@ -3559,12 +3557,12 @@
     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;
   }
@@ -3660,7 +3658,7 @@
   /**
    * @return : Returns the ConfigurationManager object for testing purposes.
    */
-  @VisibleForTesting
+  @InterfaceAudience.Private
   ConfigurationManager getConfigurationManager() {
     return configurationManager;
   }
@@ -3724,7 +3722,7 @@
    * For testing
    * @return whether all wal roll request finished for this regionserver
    */
-  @VisibleForTesting
+  @InterfaceAudience.Private
   public boolean walRollRequestFinished() {
     return this.walRoller.walRollFinished();
   }
@@ -3929,7 +3927,7 @@
     return asyncClusterConnection;
   }
 
-  @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 a05d4a6..3a71c23 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
@@ -96,8 +96,6 @@
 import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
 import org.apache.hadoop.hbase.security.EncryptionUtil;
 import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ChecksumType;
 import org.apache.hadoop.hbase.util.ClassSize;
@@ -110,7 +108,7 @@
 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;
@@ -119,6 +117,10 @@
 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.
@@ -725,7 +727,6 @@
     completeCompaction(toBeRemovedStoreFiles);
   }
 
-  @VisibleForTesting
   protected HStoreFile createStoreFileAndReader(final Path p) throws IOException {
     StoreFileInfo info = new StoreFileInfo(conf, this.getFileSystem(),
         p, isPrimaryReplicaStore());
@@ -1538,7 +1539,6 @@
     }
   }
 
-  @VisibleForTesting
   protected List<HStoreFile> doCompaction(CompactionRequestImpl cr,
       Collection<HStoreFile> filesToCompact, User user, long compactionStartTime,
       List<Path> newFiles) throws IOException {
@@ -1633,7 +1633,6 @@
         this.region.getRegionInfo(), compactionDescriptor, this.region.getMVCC());
   }
 
-  @VisibleForTesting
   void replaceStoreFiles(Collection<HStoreFile> compactedFiles, Collection<HStoreFile> result)
       throws IOException {
     this.lock.writeLock().lock();
@@ -1794,7 +1793,6 @@
    * but instead makes a compaction candidate list by itself.
    * @param N Number of files.
    */
-  @VisibleForTesting
   public void compactRecentForTestingAssumingDefaultPolicy(int N) throws IOException {
     List<HStoreFile> filesToCompact;
     boolean isMajor;
@@ -2069,7 +2067,6 @@
    * Update counts.
    * @param compactedFiles list of files that were compacted
    */
-  @VisibleForTesting
   protected void completeCompaction(Collection<HStoreFile> compactedFiles)
   // Rename this method! TODO.
     throws IOException {
@@ -2561,7 +2558,6 @@
    * Used for tests.
    * @return cache configuration for this Store.
    */
-  @VisibleForTesting
   public CacheConfig getCacheConfig() {
     return this.cacheConf;
   }
@@ -2641,7 +2637,6 @@
    * 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 5e0bf2a..7a74689 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
@@ -47,8 +47,8 @@
 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 +319,6 @@
     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 ea5586c..1f831ee 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.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 /**
  * Manages tuning of Heap memory using <code>HeapMemoryTuner</code>. 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 @@
 
   private List<HeapMemoryTuneObserver> 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 f28e28e..2fef5d1 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 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 @@
     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 cbab595..be73bba 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.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 @@
  * TODO: change to a pool of threads
  */
 @InterfaceAudience.Private
-@VisibleForTesting
 public class LogRoller extends AbstractWALRoller<RegionServerServices> {
   private static final Logger LOG = LoggerFactory.getLogger(LogRoller.class);
 
@@ -64,7 +61,6 @@
     requester.requestFlush(r, families, FlushLifeCycleTracker.DUMMY);
   }
 
-  @VisibleForTesting
   Map<WAL, RollController> 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 2dafcee..a031344 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.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 @@
     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 7951c72..f5fccf4 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.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.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.
  * <p>
@@ -73,7 +73,6 @@
   private ReentrantLock lock = new ReentrantLock();
 
   // A set of chunks contained by this memstore LAB
-  @VisibleForTesting
   Set<Integer> chunks = new ConcurrentSkipListSet<Integer>();
   private final int dataChunkSize;
   private final int maxAlloc;
@@ -270,7 +269,6 @@
     }
   }
 
-  @VisibleForTesting
   int getOpenScannerCount() {
     return this.openScannerCount.get();
   }
@@ -397,12 +395,10 @@
     return this.chunkCreator.isOffheap();
   }
 
-  @VisibleForTesting
   Chunk getCurrentChunk() {
     return currChunk.get();
   }
 
-  @VisibleForTesting
   BlockingQueue<Chunk> getPooledChunks() {
     BlockingQueue<Chunk> pooledChunks = new LinkedBlockingQueue<>();
     for (Integer id : this.chunks) {
@@ -414,7 +410,7 @@
     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 e37a272..6bbb620 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.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 /**
  * <p>
  * This class is for maintaining the various regionserver statistics
@@ -95,12 +93,10 @@
     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 2b350e6..d821eec 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 @@
     }
   }
 
-  @VisibleForTesting
   @Override
   public String toString() {
     ToStringHelper helper = MoreObjects.toStringHelper(this).add("readPoint", readPoint)
@@ -266,7 +262,6 @@
     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 2c1e859..6e81343 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.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 @@
    * 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 9efdaa9..5d9327b 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
@@ -156,7 +156,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.cache.Cache;
 import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder;
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
@@ -328,7 +327,6 @@
   final RpcServerInterface rpcServer;
   final InetSocketAddress isa;
 
-  @VisibleForTesting
   protected final HRegionServer regionServer;
   private final long maxScannerResultSize;
 
@@ -1236,7 +1234,6 @@
     }
   }
 
-  @VisibleForTesting
   public int getScannersCount() {
     return scanners.size();
   }
@@ -1374,7 +1371,6 @@
    * @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());
@@ -1401,12 +1397,10 @@
     return regions;
   }
 
-  @VisibleForTesting
   public PriorityFunction getPriority() {
     return priority;
   }
 
-  @VisibleForTesting
   public Configuration getConfiguration() {
     return regionServer.getConfiguration();
   }
@@ -3912,7 +3906,6 @@
     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 36392d7..06795a5 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.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 @@
     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 7d61618..4e20666 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.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 15d8710..e8a9154 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,8 +57,6 @@
 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;
@@ -166,12 +164,10 @@
 
   private Consumer<HRegion> fsCreatedListener;
 
-  @VisibleForTesting
   void setFsCreatedListener(Consumer<HRegion> 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 b1c92a4..b0763aa 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.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 @@
   /**
    * Get cell length after serialized in {@link KeyValue}
    */
-  @VisibleForTesting
   static int getCellLength(Cell cell) {
     return cell.getSerializedSize();
   }
@@ -414,7 +412,6 @@
     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 0b1d251..e0ea974 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.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 @@
     }
   }
 
-  @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 90e705c..ed3a73f 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
@@ -48,7 +48,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 worker is spawned in every regionserver, including master. The Worker waits for log
@@ -289,7 +288,6 @@
    * 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 f92a4d3..7550511 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.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 @@
   /**
    * 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 8c48b12..1744a7f 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.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.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 @@
   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 @@
   // Since CompactingMemstore is now default, we get three memstore scanners from a flush
   private final List<KeyValueScanner> memStoreScannersAfterFlush = new ArrayList<>(3);
   // The current list of scanners
-  @VisibleForTesting
   final List<KeyValueScanner> currentScanners = new ArrayList<>();
   // flush update lock
   private final ReentrantLock flushLock = new ReentrantLock();
@@ -344,7 +338,6 @@
   }
 
   // Used to instantiate a scanner for user scan in test
-  @VisibleForTesting
   StoreScanner(Scan scan, ScanInfo scanInfo, NavigableSet<byte[]> columns,
       List<? extends KeyValueScanner> scanners, ScanType scanType) throws IOException {
     // 0 is passed as readpoint because the test bypasses Store
@@ -361,7 +354,6 @@
   }
 
   // Used to instantiate a scanner for user scan in test
-  @VisibleForTesting
   StoreScanner(Scan scan, ScanInfo scanInfo, NavigableSet<byte[]> columns,
       List<? extends KeyValueScanner> scanners) throws IOException {
     // 0 is passed as readpoint because the test bypasses Store
@@ -373,7 +365,6 @@
   }
 
   // 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,7 +375,6 @@
     seekAllScanner(scanInfo, scanners);
   }
 
-  @VisibleForTesting
   boolean isScanUsePread() {
     return this.scanUsePread;
   }
@@ -427,7 +417,6 @@
     }
   }
 
-  @VisibleForTesting
   protected void resetKVHeap(List<? extends KeyValueScanner> scanners,
       CellComparator comparator) throws IOException {
     // Combine all seeked scanners with a heap
@@ -444,7 +433,6 @@
    * <p>
    * Will be overridden by testcase so declared as protected.
    */
-  @VisibleForTesting
   protected List<KeyValueScanner> selectScannersFrom(HStore store,
       List<? extends KeyValueScanner> allScanners) {
     boolean memOnly;
@@ -870,7 +858,6 @@
    * @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
@@ -896,7 +883,6 @@
    * @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
@@ -1096,7 +1082,6 @@
     return heap.reseek(kv);
   }
 
-  @VisibleForTesting
   void trySwitchToStreamRead() {
     if (readType != Scan.ReadType.DEFAULT || !scanUsePread || closing ||
         heap.peek() == null || bytesRead < preadMaxBytes) {
@@ -1208,7 +1193,6 @@
    * Used in testing.
    * @return all scanners in no particular order
    */
-  @VisibleForTesting
   List<KeyValueScanner> getAllScannersForTesting() {
     List<KeyValueScanner> 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 7beec5e..1560aef 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.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 @@
       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 37923ad..fdf9db2 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.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 @@
     return TimeRange.between(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 @@
     }
   }
 
-  @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 2a684da..d5be356 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 @@
 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 @@
   }
 
   // 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 b683634..1f5a820 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 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 @@
     }
   }
 
-  @VisibleForTesting
   static Tick nextTick() {
     Calendar calendar = new GregorianCalendar();
     calendar.setTimeInMillis(EnvironmentEdgeManager.currentTime());
@@ -54,7 +52,6 @@
     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 1cc7dda..f60e97d 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.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.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;
@@ -98,7 +97,7 @@
    * Heuristics for guessing whether we need minor compaction.
    */
   @Override
-  @VisibleForTesting
+  @InterfaceAudience.Private
   public boolean needsCompaction(Collection<HStoreFile> storeFiles,
       List<HStoreFile> filesCompacting) {
     ArrayList<HStoreFile> 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 ee6db31..b907aa6 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
@@ -34,8 +34,6 @@
 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
@@ -184,7 +182,6 @@
     return this.parallelPutToStoreThreadLimit > 0;
   }
 
-  @VisibleForTesting
   Map<byte[], AtomicInteger> 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 ac99ea6..af5bfd5 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.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;
@@ -87,13 +86,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.util.concurrent.ThreadFactoryBuilder;
 
-
-
-
-
 /**
  * Implementation of {@link WAL} to go against {@link FileSystem}; i.e. keep WALs in HDFS. Only one
  * WAL is ever being written at a time. When a WAL hits a configured maximum size, it is rolled.
@@ -610,7 +605,6 @@
     return newPath;
   }
 
-  @VisibleForTesting
   Path getOldPath() {
     long currentFilenum = this.filenum.get();
     Path oldPath = null;
@@ -785,7 +779,6 @@
     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.
@@ -837,7 +830,6 @@
    * @return the passed in <code>newPath</code>
    * @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);
@@ -941,7 +933,6 @@
    * 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);
   }
@@ -1040,7 +1031,6 @@
   /**
    * Exposed for testing only. Use to tricks like halt the ring buffer appending.
    */
-  @VisibleForTesting
   protected void atHeadOfRingBufferEventHandlerAppend() {
     // Noop
   }
@@ -1295,13 +1285,11 @@
   /**
    * 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/DualAsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java
index bf5b96d..1279c2f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java
@@ -29,7 +29,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
 import org.apache.hbase.thirdparty.io.netty.channel.Channel;
 import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
@@ -61,7 +60,6 @@
   }
 
   // will be overridden in testcase
-  @VisibleForTesting
   protected AsyncWriter createCombinedAsyncWriter(AsyncWriter localWriter,
       AsyncWriter remoteWriter) {
     return CombinedAsyncWriter.create(remoteWriter, localWriter);
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 fe910aa..e2320db 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.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;
@@ -64,9 +63,8 @@
 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 +202,11 @@
    * @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,
@@ -283,7 +279,6 @@
    * removed.
    * @return null if underlying stream is not ready.
    */
-  @VisibleForTesting
   OutputStream getOutputStream() {
     FSDataOutputStream fsdos = this.hdfs_out;
     return fsdos != null ? fsdos.getWrappedStream() : null;
@@ -321,14 +316,12 @@
    * Used to manufacture race condition reliably. For testing only.
    * @see #beforeWaitOnSafePoint()
    */
-  @VisibleForTesting
   protected void afterCreatingZigZagLatch() {
   }
 
   /**
    * @see #afterCreatingZigZagLatch()
    */
-  @VisibleForTesting
   protected void beforeWaitOnSafePoint() {
   }
 
@@ -777,7 +770,6 @@
     return logRollNeeded;
   }
 
-  @VisibleForTesting
   protected long getSequenceOnRingBuffer() {
     return this.disruptor.getRingBuffer().next();
   }
@@ -787,7 +779,6 @@
     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);
@@ -814,7 +805,6 @@
    * patch.
    */
   @Override
-  @VisibleForTesting
   int getLogReplication() {
     try {
       // in standalone mode, it will return 0
@@ -855,7 +845,6 @@
     }
   }
 
-  @VisibleForTesting
   boolean isLowReplicationRollEnabled() {
     return lowReplicationRollEnabled;
   }
@@ -1211,12 +1200,10 @@
     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 44c96de..ca51ec0 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.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 @@
     }
   }
 
-  @VisibleForTesting
   static Set<byte[]> collectFamilies(List<Cell> 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 ee99aa3..f23dae2 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,18 +19,16 @@
 
 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.CompatibilitySingletonFactory;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.wal.WALEdit;
-import org.apache.hadoop.hbase.wal.WALKey;
-import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
-import org.apache.hadoop.util.StringUtils;
 
 /**
  * Class used to push numbers about the WAL into the metrics subsystem.  This will take a
@@ -46,7 +44,6 @@
     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 7146ca7..5c6fcd2 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
@@ -37,8 +37,6 @@
 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
@@ -240,7 +238,6 @@
     }
   }
 
-  @VisibleForTesting
   ConcurrentMap<ImmutableByteArray, Long> 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/ChainWALEmptyEntryFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ChainWALEmptyEntryFilter.java
index 19fd0c7..4923647 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ChainWALEmptyEntryFilter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ChainWALEmptyEntryFilter.java
@@ -21,7 +21,6 @@
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 /**
  * A {@link ChainWALEntryFilter} for providing more flexible options
@@ -55,7 +54,7 @@
    *
    * @param filterEmptyEntry flag
    */
-  @VisibleForTesting
+  @InterfaceAudience.Private
   public void setFilterEmptyEntry(final boolean filterEmptyEntry) {
     this.filterEmptyEntry = filterEmptyEntry;
   }
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 b08c990..8678685 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
@@ -25,7 +25,6 @@
 import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.ThreadLocalRandom;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.client.AsyncClusterConnection;
@@ -33,12 +32,12 @@
 import org.apache.hadoop.hbase.client.ClusterConnectionFactory;
 import org.apache.hadoop.hbase.security.User;
 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;
@@ -46,7 +45,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
 
 /**
@@ -312,7 +310,6 @@
     badReportCounts.remove(sinkPeer.getServerName());
   }
 
-  @VisibleForTesting
   List<ServerName> getSinkServers() {
     return sinkServers;
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationBarrierFamilyFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationBarrierFamilyFormat.java
index 123a036..81be5a3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationBarrierFamilyFormat.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationBarrierFamilyFormat.java
@@ -49,8 +49,6 @@
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.yetus.audience.InterfaceAudience;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 /**
  * Helper class for storing replication barriers in family 'rep_barrier' of meta table.
  * <p/>
@@ -59,7 +57,6 @@
 @InterfaceAudience.Private
 public final class ReplicationBarrierFamilyFormat {
 
-  @VisibleForTesting
   public static final byte[] REPLICATION_PARENT_QUALIFIER = Bytes.toBytes("parent");
 
   private static final byte ESCAPE_BYTE = (byte) 0xFF;
@@ -85,7 +82,6 @@
     }
   }
 
-  @VisibleForTesting
   public static byte[] getParentsBytes(List<RegionInfo> parents) {
     ByteArrayOutputStream bos = new ByteArrayOutputStream();
     Iterator<RegionInfo> iter = parents.iterator();
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 3271696..f06b29c 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.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 @@
     }
   }
 
-  @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 4e0be94..8f016bc 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
@@ -20,7 +20,6 @@
 import java.io.IOException;
 import java.util.Collections;
 import java.util.Set;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
@@ -34,7 +33,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.base.Predicate;
 import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
 
@@ -105,7 +103,7 @@
     }
   }
 
-  @VisibleForTesting
+  @InterfaceAudience.Private
   public void setConf(Configuration conf, ZKWatcher zk) {
     super.setConf(conf);
     try {
@@ -115,7 +113,8 @@
       LOG.error("Error while configuring " + this.getClass().getName(), e);
     }
   }
-  @VisibleForTesting
+
+  @InterfaceAudience.Private
   public void setConf(Configuration conf, ZKWatcher zk,
       ReplicationQueueStorage replicationQueueStorage) {
     super.setConf(conf);
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 56bccc0..c77f74f 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.TimeUnit;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
-
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Abortable;
@@ -66,7 +65,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 /**
@@ -263,7 +261,6 @@
   /**
    * 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();
@@ -282,7 +279,6 @@
   /**
    * 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();
@@ -298,7 +294,6 @@
     return false;
   }
 
-  @VisibleForTesting
   List<List<Entry>> filterNotExistTableEdits(final List<List<Entry>> oldEntryList) {
     List<List<Entry>> entryList = new ArrayList<>();
     Map<TableName, Boolean> existMap = new HashMap<>();
@@ -342,7 +337,6 @@
     return entryList;
   }
 
-  @VisibleForTesting
   List<List<Entry>> filterNotExistColumnFamilyEdits(final List<List<Entry>> oldEntryList) {
     List<List<Entry>> entryList = new ArrayList<>();
     Map<TableName, Set<String>> existColumnFamilyMap = new HashMap<>();
@@ -556,7 +550,6 @@
     notifyStopped();
   }
 
-  @VisibleForTesting
   protected int replicateEntries(List<Entry> entries, int batchIndex, int timeout)
       throws IOException {
     SinkPeer sinkPeer = null;
@@ -612,7 +605,6 @@
     return batchIndex;
   }
 
-  @VisibleForTesting
   protected Callable<Integer> createReplicator(List<Entry> 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 0f73576..098418d 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
@@ -32,8 +32,6 @@
 import org.apache.hadoop.hbase.metrics.BaseSource;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 /**
  * This class is for maintaining the various replication statistics for a source and publishing them
  * through the metrics interfaces.
@@ -450,7 +448,7 @@
     return globalSourceSource.getMetricsName();
   }
 
-  @VisibleForTesting
+  @InterfaceAudience.Private
   public Map<String, MetricsReplicationTableSource> getSingleSourceSourceByTable() {
     return singleSourceSourceByTable;
   }
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 9e5fb94..423ec0e 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
@@ -67,7 +67,7 @@
 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;
 
 /**
@@ -140,7 +140,6 @@
   private long defaultBandwidth;
   private long currentBandwidth;
   private WALFileLengthProvider walFileLengthProvider;
-  @VisibleForTesting
   protected final ConcurrentHashMap<String, ReplicationSourceShipper> workerThreads =
       new ConcurrentHashMap<>();
 
@@ -455,7 +454,6 @@
    * 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 e323304..95157ca 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
@@ -77,7 +77,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.common.collect.Sets;
 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 
@@ -197,7 +197,6 @@
    * 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<ReplicationSourceInterface> catalogReplicationSource = new AtomicReference<>();
 
   /**
@@ -395,7 +394,6 @@
    * @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);
@@ -673,7 +671,6 @@
    * @param inclusive whether we should also remove the given log file
    * @param source the replication source
    */
-  @VisibleForTesting
   void cleanOldLogs(String log, boolean inclusive, ReplicationSourceInterface source) {
     String logPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(log);
     if (source.isRecovered()) {
@@ -777,7 +774,6 @@
   }
 
   // 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);
@@ -827,7 +823,6 @@
   }
 
   // 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()) {
@@ -879,7 +874,6 @@
     // the rs will abort (See HBASE-20475).
     private final Map<String, ReplicationPeerImpl> peersSnapshot;
 
-    @VisibleForTesting
     public NodeFailoverWorker(ServerName deadRS) {
       super("Failover-for-" + deadRS);
       this.deadRS = deadRS;
@@ -1034,7 +1028,6 @@
    * Get a copy of the wals of the normal sources on this rs
    * @return a sorted set of wal names
    */
-  @VisibleForTesting
   public Map<String, Map<String, NavigableSet<String>>> getWALs() {
     return Collections.unmodifiableMap(walsById);
   }
@@ -1043,7 +1036,6 @@
    * Get a copy of the wals of the recovered sources on this rs
    * @return a sorted set of wal names
    */
-  @VisibleForTesting
   Map<String, Map<String, NavigableSet<String>>> getWalsByIdRecoveredQueues() {
     return Collections.unmodifiableMap(walsByIdRecoveredQueues);
   }
@@ -1068,12 +1060,10 @@
    * 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<String> getAllQueues() throws IOException {
     List<String> allQueues = Collections.emptyList();
     try {
@@ -1084,21 +1074,18 @@
     return allQueues;
   }
 
-  @VisibleForTesting
   int getSizeOfLatestPath() {
     synchronized (latestPaths) {
       return latestPaths.size();
     }
   }
 
-  @VisibleForTesting
   Set<Path> getLastestPath() {
     synchronized (latestPaths) {
       return Sets.newHashSet(latestPaths.values());
     }
   }
 
-  @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 d25ab07..a38a96a 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.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 @@
    * @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/rsgroup/RSGroupBasedLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java
index 27d407e..89db769 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java
@@ -26,7 +26,6 @@
 import java.util.Map;
 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.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;
@@ -416,7 +414,6 @@
     return false;
   }
 
-  @VisibleForTesting
   public void setRsGroupInfoManager(RSGroupInfoManager rsGroupInfoManager) {
     this.rsGroupInfoManager = rsGroupInfoManager;
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
index 16a44ad..bae2ba5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
@@ -24,7 +24,6 @@
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -90,7 +89,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
@@ -102,6 +100,7 @@
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MultiRowMutationProtos.MutateRowsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupProtos;
+
 /**
  * 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
@@ -126,31 +125,24 @@
   private static final Logger LOG = LoggerFactory.getLogger(RSGroupInfoManagerImpl.class);
 
   // Assigned before user tables
-  @VisibleForTesting
   static final TableName RSGROUP_TABLE_NAME =
       TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "rsgroup");
 
-  @VisibleForTesting
   static final String KEEP_ONE_SERVER_IN_DEFAULT_ERROR_MESSAGE = "should keep at least " +
       "one server in 'default' RSGroup.";
 
   /** Define the config key of retries threshold when movements failed */
-  @VisibleForTesting
   static final String FAILED_MOVE_MAX_RETRY = "hbase.rsgroup.move.max.retry";
 
   /** Define the default number of retries */
-  @VisibleForTesting
   static final int DEFAULT_MAX_RETRY_VALUE = 50;
 
   private static final String RS_GROUP_ZNODE = "rsgroup";
 
-  @VisibleForTesting
   static final byte[] META_FAMILY_BYTES = Bytes.toBytes("m");
 
-  @VisibleForTesting
   static final byte[] META_QUALIFIER_BYTES = Bytes.toBytes("i");
 
-  @VisibleForTesting
   static final String MIGRATE_THREAD_NAME = "Migrate-RSGroup-Tables";
 
   private static final byte[] ROW_KEY = { 0 };
@@ -1119,7 +1111,6 @@
    * parent region cases. This method is invoked by {@link #balanceRSGroup}
    * @return A clone of current assignments for this group.
    */
-  @VisibleForTesting
   Map<TableName, Map<ServerName, List<RegionInfo>>> getRSGroupAssignmentsByTable(
     TableStateManager tableStateManager, String groupName) throws IOException {
     Map<TableName, Map<ServerName, List<RegionInfo>>> result = Maps.newHashMap();
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupMajorCompactionTTL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupMajorCompactionTTL.java
index 172561d..aec38ee 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupMajorCompactionTTL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupMajorCompactionTTL.java
@@ -32,7 +32,6 @@
 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;
@@ -49,7 +48,7 @@
 
   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/security/access/PermissionStorage.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/PermissionStorage.java
index 1006e54..268bc36 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
@@ -70,7 +70,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap;
 import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap;
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
@@ -501,7 +500,6 @@
       null, false);
   }
 
-  @VisibleForTesting
   public static ListMultimap<String, UserPermission> getNamespacePermissions(Configuration conf,
       String namespace) throws IOException {
     return getPermissions(conf, Bytes.toBytes(toNamespaceEntry(namespace)), null, null, null, null,
@@ -780,7 +778,6 @@
     return list;
   }
 
-  @VisibleForTesting
   public static ListMultimap<String, UserPermission> 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 8ce4266..72da07c 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.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.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 @@
     }
   }
 
-  @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 @@
     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 e52134e..acb6940 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.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.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;
 
 /**
@@ -542,7 +540,7 @@
     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 50f8784..2398ba4 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.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.ZKWatcher;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 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 @@
    * 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 9df33e1..5e82cad 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.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 @@
     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 @@
     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 @@
     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/BulkLoadHFilesTool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java
index a39ecaf..ec9a59c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java
@@ -94,16 +94,15 @@
 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;
 
 /**
  * The implementation for {@link BulkLoadHFiles}, and also can be executed from command line as a
@@ -386,7 +385,7 @@
    * @param lqis hfiles should be loaded
    * @return empty list if success, list of items to retry on recoverable failure
    */
-  @VisibleForTesting
+  @InterfaceAudience.Private
   protected CompletableFuture<Collection<LoadQueueItem>> tryAtomicRegionLoad(
       final AsyncClusterConnection conn, final TableName tableName, boolean copyFiles,
       final byte[] first, Collection<LoadQueueItem> lqis) {
@@ -434,7 +433,7 @@
    * <p/>
    * protected for testing.
    */
-  @VisibleForTesting
+  @InterfaceAudience.Private
   protected void bulkLoadPhase(AsyncClusterConnection conn, TableName tableName,
       Deque<LoadQueueItem> queue, Multimap<ByteBuffer, LoadQueueItem> regionGroups,
       boolean copyFiles, Map<LoadQueueItem, ByteBuffer> item2RegionMap) throws IOException {
@@ -662,7 +661,7 @@
    * protected for testing
    * @throws IOException if an IO failure is encountered
    */
-  @VisibleForTesting
+  @InterfaceAudience.Private
   protected Pair<List<LoadQueueItem>, String> groupOrSplit(AsyncClusterConnection conn,
       TableName tableName, Multimap<ByteBuffer, LoadQueueItem> regionGroups, LoadQueueItem item,
       List<Pair<byte[], byte[]>> startEndKeys) throws IOException {
@@ -717,7 +716,7 @@
    * 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/Canary.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
index b49d2ce..89ad398 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 @@
 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 @@
     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 59e0e6c..f2c2d46 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.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 @@
     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/coprocessor/CoprocessorValidator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/coprocessor/CoprocessorValidator.java
index 0e76f0e..766224e 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.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.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 @@
     validateClasses(classLoader, Arrays.asList(classNames), violations);
   }
 
-  @VisibleForTesting
+  @InterfaceAudience.Private
   protected void validateTables(ClassLoader classLoader, Admin admin,
       Pattern pattern, List<CoprocessorViolation> violations) throws IOException {
     List<TableDescriptor> 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 b932c4d..c7afb0e 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.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 @@
  *   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 937069f..4fb2312 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.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 @@
   private final boolean usecache;
   private volatile boolean fsvisited;
 
-  @VisibleForTesting
   long cachehits = 0;
-  @VisibleForTesting
   long invocations = 0;
 
   /**
@@ -117,7 +114,6 @@
     this.usecache = usecache;
   }
 
-  @VisibleForTesting
   public static void tryUpdateMetaTableDescriptor(Configuration conf) throws IOException {
     tryUpdateAndGetMetaTableDescriptor(conf, CommonFSUtils.getCurrentFileSystem(conf),
       CommonFSUtils.getRootDir(conf));
@@ -192,7 +188,6 @@
         .setPriority(Coprocessor.PRIORITY_SYSTEM).build());
   }
 
-  @VisibleForTesting
   protected boolean isUsecache() {
     return this.usecache;
   }
@@ -299,7 +294,6 @@
     }
   }
 
-  @VisibleForTesting
   Path updateTableDescriptor(TableDescriptor td) throws IOException {
     TableName tableName = td.getTableName();
     Path tableDir = getTableDir(tableName);
@@ -410,7 +404,6 @@
   /**
    * Compare {@link FileStatus} instances by {@link Path#getName()}. Returns in reverse order.
    */
-  @VisibleForTesting
   static final Comparator<FileStatus> TABLEINFO_FILESTATUS_COMPARATOR =
     new Comparator<FileStatus>() {
       @Override
@@ -422,7 +415,6 @@
   /**
    * Return the table directory in HDFS
    */
-  @VisibleForTesting
   Path getTableDir(final TableName tableName) {
     return CommonFSUtils.getTableDir(rootdir, tableName);
   }
@@ -437,7 +429,6 @@
   /**
    * Width of the sequenceid that is a suffix on a tableinfo file.
    */
-  @VisibleForTesting
   static final int WIDTH_OF_SEQUENCE_ID = 10;
 
   /**
@@ -467,7 +458,6 @@
    * @param p Path to a <code>.tableinfo</code> file.
    * @return The current editid or 0 if none found.
    */
-  @VisibleForTesting
   static int getTableInfoSequenceId(final Path p) {
     if (p == null) {
       return 0;
@@ -487,7 +477,6 @@
    * @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 31e7d3f..413b6ba 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
@@ -83,16 +83,15 @@
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.util.Progressable;
-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.base.Throwables;
 import org.apache.hbase.thirdparty.com.google.common.collect.Iterators;
 import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 import org.apache.hbase.thirdparty.com.google.common.primitives.Ints;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos;
@@ -108,7 +107,7 @@
   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 a52f3fb..4ea4c9e 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
@@ -134,6 +134,10 @@
 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;
@@ -141,12 +145,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-import org.apache.hbase.thirdparty.com.google.common.base.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;
-
 /**
  * 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
@@ -213,7 +211,7 @@
    * 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
@@ -400,7 +398,7 @@
   /**
    * @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 8b05fa9..10cc4e9 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
@@ -21,7 +21,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
@@ -43,7 +42,6 @@
 public abstract class IdReadWriteLock<T> {
   public abstract ReentrantReadWriteLock getLock(T id);
 
-  @VisibleForTesting
   public void waitForWaiters(T id, int numWaiters) throws InterruptedException {
     for (ReentrantReadWriteLock readWriteLock;;) {
       readWriteLock = getLock(id);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/IdReadWriteLockStrongRef.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/IdReadWriteLockStrongRef.java
index e119784..179b7d4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/IdReadWriteLockStrongRef.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/IdReadWriteLockStrongRef.java
@@ -22,14 +22,12 @@
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 @InterfaceAudience.Private
 public class IdReadWriteLockStrongRef<T> extends IdReadWriteLock<T> {
 
   final ConcurrentHashMap<T, ReentrantReadWriteLock> map = new ConcurrentHashMap<>();
 
-  @VisibleForTesting
   @Override
   public ReentrantReadWriteLock getLock(T id) {
     ReentrantReadWriteLock existing = map.get(id);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/IdReadWriteLockWithObjectPool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/IdReadWriteLockWithObjectPool.java
index 130ad14..5492a85 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/IdReadWriteLockWithObjectPool.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/IdReadWriteLockWithObjectPool.java
@@ -23,8 +23,6 @@
 
 import org.apache.yetus.audience.InterfaceAudience;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 @InterfaceAudience.Private
 public class IdReadWriteLockWithObjectPool<T> extends IdReadWriteLock<T>{
   // The number of lock we want to easily support. It's not a maximum.
@@ -84,7 +82,6 @@
   }
 
   /** For testing */
-  @VisibleForTesting
   int purgeAndGetEntryPoolSize() {
     gc();
     Threads.sleep(200);
@@ -97,7 +94,6 @@
     System.gc();
   }
 
-  @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 9d7cb56..9ade12d 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.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 @@
   /**
    * sweep low frequency data
    */
-  @VisibleForTesting
   public void sweep() {
     for(Map.Entry<T, Integer> entry : data.entrySet()) {
       if(entry.getValue() < currentTerm) {
@@ -168,7 +167,7 @@
     }
   }
 
-  @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 03e674d..2f7d351 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
@@ -66,7 +66,6 @@
 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;
 
@@ -141,7 +140,7 @@
     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;
 
@@ -385,8 +384,8 @@
     return waitTaskToFinish(unloadPool, unloadTask, "unloading");
   }
 
-  @VisibleForTesting
-   Collection<ServerName> filterRSGroupServers(RSGroupInfo rsgroup,
+  @InterfaceAudience.Private
+  Collection<ServerName> filterRSGroupServers(RSGroupInfo rsgroup,
       Collection<ServerName> onlineServers) {
     if (rsgroup.getName().equals(RSGroupInfo.DEFAULT_GROUP)) {
       return onlineServers;
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 c0d34d9..9be182d 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.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 @@
     }
   }
 
-  @VisibleForTesting List<MajorCompactionRequest> getQueue(ServerName serverName) {
+  List<MajorCompactionRequest> 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 aecfc37..22ec6cb 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.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 @@
     this.region = region;
   }
 
-  @VisibleForTesting
   MajorCompactionRequest(Configuration configuration, RegionInfo region,
       Set<String> stores) {
     this(configuration, region);
@@ -81,7 +79,6 @@
     this.stores = stores;
   }
 
-  @VisibleForTesting
   Optional<MajorCompactionRequest> createRequest(Configuration configuration,
       Set<String> stores, long timestamp) throws IOException {
     Set<String> familiesToCompact = getStoresRequiringCompaction(stores, timestamp);
@@ -145,7 +142,6 @@
     return false;
   }
 
-  @VisibleForTesting
   Connection getConnection(Configuration configuration) throws IOException {
     return ConnectionFactory.createConnection(configuration);
   }
@@ -166,13 +162,11 @@
 
   }
 
-  @VisibleForTesting
   List<Path> getReferenceFilePaths(FileSystem fileSystem, Path familyDir)
       throws IOException {
     return FSUtils.getReferenceFilePaths(fileSystem, familyDir);
   }
 
-  @VisibleForTesting
   HRegionFileSystem getFileSystem(Connection connection) throws IOException {
     Admin admin = connection.getAdmin();
     return HRegionFileSystem.openRegionFromFileSystem(admin.getConfiguration(),
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionTTLRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionTTLRequest.java
index 4d2b341..0eda459 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionTTLRequest.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionTTLRequest.java
@@ -22,7 +22,6 @@
 import java.util.Collection;
 import java.util.Map;
 import java.util.Optional;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
@@ -34,7 +33,7 @@
 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;
 
 /**
@@ -55,7 +54,6 @@
     return request.createRequest(conf, htd);
   }
 
-  @VisibleForTesting
   private Optional<MajorCompactionRequest> createRequest(Configuration conf, TableDescriptor htd)
       throws IOException {
     Map<String, Long> familiesToCompact = getStoresRequiringCompaction(htd);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactor.java
index 151b492..370a3e8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactor.java
@@ -29,7 +29,6 @@
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.hbase.HBaseConfiguration;
@@ -50,7 +49,7 @@
 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.Joiner;
 import org.apache.hbase.thirdparty.com.google.common.base.Splitter;
 import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
@@ -156,7 +155,8 @@
     LOG.info("All regions major compacted successfully");
   }
 
-  @VisibleForTesting void initializeWorkQueues() throws IOException {
+  @InterfaceAudience.Private
+  void initializeWorkQueues() throws IOException {
     if (storesToCompact.isEmpty()) {
       connection.getTable(tableName).getDescriptor().getColumnFamilyNames()
           .forEach(a -> storesToCompact.add(Bytes.toString(a)));
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactorTTL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactorTTL.java
index 321cbe0..7c92c53 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactorTTL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactorTTL.java
@@ -23,7 +23,6 @@
 import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.Executors;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
@@ -38,7 +37,7 @@
 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 org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLineParser;
@@ -58,7 +57,7 @@
 
   private TableDescriptor htd;
 
-  @VisibleForTesting
+  @InterfaceAudience.Private
   public MajorCompactorTTL(Configuration conf, TableDescriptor htd, int concurrency,
       long sleepForMs) throws IOException {
     this.connection = ConnectionFactory.createConnection(conf);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
index 109e110..720e2c2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
@@ -30,7 +30,6 @@
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -49,7 +48,7 @@
 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.Lists;
 
 /**
@@ -217,7 +216,6 @@
   /**
    * returns the number of rolled WAL files.
    */
-  @VisibleForTesting
   public static int getNumRolledLogFiles(WAL wal) {
     return ((AbstractFSWAL<?>) wal).getNumRolledLogFiles();
   }
@@ -225,7 +223,6 @@
   /**
    * returns the size of rolled WAL files.
    */
-  @VisibleForTesting
   public static long getLogFileSize(WAL wal) {
     return ((AbstractFSWAL<?>) wal).getLogFileSize();
   }
@@ -233,7 +230,6 @@
   /**
    * return the current filename from the current wal.
    */
-  @VisibleForTesting
   public static Path getCurrentFileName(final WAL wal) {
     return ((AbstractFSWAL<?>) wal).getCurrentFileName();
   }
@@ -241,7 +237,6 @@
   /**
    * request a log roll, but don't actually do it.
    */
-  @VisibleForTesting
   static void requestLogRoll(final WAL wal) {
     ((AbstractFSWAL<?>) wal).requestLogRoll();
   }
@@ -249,7 +244,6 @@
   // should be package private; more visible for use in AbstractFSWAL
   public static final String WAL_FILE_NAME_DELIMITER = ".";
   /** The hbase:meta region's WAL filename extension */
-  @VisibleForTesting
   public static final String META_WAL_PROVIDER_ID = ".meta";
   static final String DEFAULT_PROVIDER_ID = "default";
 
@@ -275,7 +269,6 @@
    * @param wal must not be null
    * @return the file number that is part of the WAL file name
    */
-  @VisibleForTesting
   public static long extractFileNumFromWAL(final WAL wal) {
     final Path walPath = ((AbstractFSWAL<?>) wal).getCurrentFileName();
     if (walPath == null) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/EntryBuffers.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/EntryBuffers.java
index 6348e5c..0ca1219 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/EntryBuffers.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/EntryBuffers.java
@@ -34,8 +34,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 /**
  * Class which accumulates edits and separates them into a buffer per region while simultaneously
  * accounting RAM usage. Blocks if the RAM usage crosses a predefined threshold. Writer threads then
@@ -129,7 +127,6 @@
     }
   }
 
-  @VisibleForTesting
   synchronized boolean isRegionCurrentlyWriting(byte[] region) {
     return currentlyWriting.contains(region);
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java
index 001e1a8..f57ec31 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java
@@ -35,7 +35,6 @@
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.TableName;
@@ -54,7 +53,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
 import org.apache.hbase.thirdparty.com.google.common.collect.Streams;
 import org.apache.hbase.thirdparty.io.netty.channel.Channel;
@@ -73,7 +71,6 @@
   private static final Logger LOG = LoggerFactory.getLogger(SyncReplicationWALProvider.class);
 
   // only for injecting errors for testcase, do not use it for other purpose.
-  @VisibleForTesting
   public static final String DUAL_WAL_IMPL = "hbase.wal.sync.impl";
 
   private final WALProvider provider;
@@ -347,7 +344,6 @@
     }
   }
 
-  @VisibleForTesting
   WALProvider getWrappedProvider() {
     return provider;
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
index 747b277..2a434a7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
@@ -31,8 +31,6 @@
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 /**
  * A Write Ahead Log (WAL) provides service for reading, writing waledits. This interface provides
  * APIs for WAL users (such as RegionServer) to use the WAL (do append, sync, etc).
@@ -221,7 +219,6 @@
    * @deprecated Since version 1.2.0. Removing because not used and exposes subtle internal
    * workings. Use {@link #getEarliestMemStoreSeqNum(byte[], byte[])}
    */
-  @VisibleForTesting
   @Deprecated
   long getEarliestMemStoreSeqNum(byte[] encodedRegionName);
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java
index 1fae7a5..61f36fa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java
@@ -36,8 +36,6 @@
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.yetus.audience.InterfaceAudience;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 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;
@@ -93,14 +91,14 @@
    *   {@link #isCompactionMarker(Cell)}
    */
   @Deprecated
-  @VisibleForTesting
+  @InterfaceAudience.Private
   public static final byte[] COMPACTION = Bytes.toBytes("HBASE::COMPACTION");
 
   /**
    * @deprecated Since 2.3.0. Make it protected, internal-use only.
    */
   @Deprecated
-  @VisibleForTesting
+  @InterfaceAudience.Private
   public static final byte [] FLUSH = Bytes.toBytes("HBASE::FLUSH");
 
   /**
@@ -130,7 +128,7 @@
   private static final byte [] REGION_EVENT_CLOSE =
       createRegionEventDescriptorQualifier(RegionEventDescriptor.EventType.REGION_CLOSE);
 
-  @VisibleForTesting
+  @InterfaceAudience.Private
   public static final byte [] BULK_LOAD = Bytes.toBytes("HBASE::BULK_LOAD");
 
   private final transient boolean replay;
@@ -332,7 +330,7 @@
     return createRegionEventWALEdit(getRowForRegion(hri), regionEventDesc);
   }
 
-  @VisibleForTesting
+  @InterfaceAudience.Private
   public static WALEdit createRegionEventWALEdit(byte [] rowForRegion,
       RegionEventDescriptor regionEventDesc) {
     KeyValue kv = new KeyValue(rowForRegion, METAFAMILY,
@@ -345,7 +343,7 @@
    * @return Cell qualifier for the passed in RegionEventDescriptor Type; e.g. we'll
    *   return something like a byte array with HBASE::REGION_EVENT::REGION_OPEN in it.
    */
-  @VisibleForTesting
+  @InterfaceAudience.Private
   public static byte [] createRegionEventDescriptorQualifier(RegionEventDescriptor.EventType t) {
     return Bytes.toBytes(REGION_EVENT_PREFIX_STR + t.toString());
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
index 6964bb8..82e974d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
@@ -37,7 +37,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;
 
 /**
  * Entry point for users of the Write Ahead Log.
@@ -124,12 +123,10 @@
     this.abortable = null;
   }
 
-  @VisibleForTesting
   Providers getDefaultProvider() {
     return Providers.defaultProvider;
   }
 
-  @VisibleForTesting
   public Class<? extends WALProvider> getProviderClass(String key, String defaultValue) {
     try {
       Providers provider = Providers.valueOf(conf.get(key, defaultValue));
@@ -403,7 +400,6 @@
    * Uses defaults.
    * @return an overwritable writer for recovered edits. caller should close.
    */
-  @VisibleForTesting
   public Writer createRecoveredEditsWriter(final FileSystem fs, final Path path)
       throws IOException {
     return FSHLogProvider.createWriter(conf, fs, path, true);
@@ -483,7 +479,6 @@
    * Uses defaults.
    * @return a writer that won't overwrite files. Caller must close.
    */
-  @VisibleForTesting
   public static Writer createWALWriter(final FileSystem fs, final Path path,
       final Configuration configuration)
       throws IOException {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java
index 33e0343..4c3fc4e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java
@@ -25,7 +25,6 @@
 import java.util.NavigableMap;
 import java.util.TreeMap;
 import java.util.UUID;
-
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
@@ -37,7 +36,6 @@
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.yetus.audience.InterfaceAudience;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
@@ -130,7 +128,7 @@
         new ArrayList<>(), HConstants.NO_NONCE, HConstants.NO_NONCE, null, replicationScope, null);
   }
 
-  @VisibleForTesting
+  @InterfaceAudience.Private
   public WALKeyImpl(final byte[] encodedRegionName, final TableName tablename, long logSeqNum,
       final long now, UUID clusterId) {
     List<UUID> clusterIds = new ArrayList<>(1);
@@ -139,7 +137,7 @@
       HConstants.NO_NONCE, null, null, null);
   }
 
-  @VisibleForTesting
+  @InterfaceAudience.Private
   public WALKeyImpl(final byte[] encodedRegionName, final TableName tablename, long logSeqNum,
       final long now, UUID clusterId, MultiVersionConcurrencyControl mvcc) {
     List<UUID> clusterIds = new ArrayList<>(1);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitUtil.java
index d392366..6361ffc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitUtil.java
@@ -16,6 +16,7 @@
  * limitations under the License.
  */
 package org.apache.hadoop.hbase.wal;
+
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.ArrayList;
@@ -56,7 +57,7 @@
 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.AdminProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
@@ -157,7 +158,6 @@
    * @return Path to file into which to dump split log edits.
    */
   @SuppressWarnings("deprecation")
-  @VisibleForTesting
   static Path getRegionSplitEditsPath(TableName tableName, byte[] encodedRegionName, long seqId,
       String fileNameBeingSplit, String tmpDirName, Configuration conf) throws IOException {
     FileSystem walFS = CommonFSUtils.getWALFileSystem(conf);
@@ -207,7 +207,6 @@
     return new Path(srcPath.getParent(), fileName);
   }
 
-  @VisibleForTesting
   static String formatRecoveredEditsFileName(final long seqid) {
     return String.format("%019d", seqid);
   }
@@ -345,7 +344,6 @@
   /**
    * Is the given file a region open sequence id file.
    */
-  @VisibleForTesting
   public static boolean isSequenceIdFile(final Path file) {
     return file.getName().endsWith(SEQUENCE_ID_FILE_SUFFIX)
         || file.getName().endsWith(OLD_SEQUENCE_ID_FILE_SUFFIX);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
index 0bea35e..6616a11 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
@@ -29,7 +29,6 @@
 import java.util.TreeMap;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicReference;
-import edu.umd.cs.findbugs.annotations.Nullable;
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -57,12 +56,12 @@
 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.TextFormat;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
-import javax.validation.constraints.Null;
 
 /**
  * Split RegionServer WAL files. Splits the WAL into new files,
@@ -144,7 +143,6 @@
     this(factory, conf, walRootDir, walFS, rootDir, rootFS, null, null, null);
   }
 
-  @VisibleForTesting
   WALSplitter(final WALFactory factory, Configuration conf, Path walRootDir,
       FileSystem walFS, Path rootDir, FileSystem rootFS, LastSequenceId idChecker,
       SplitLogWorkerCoordination splitLogWorkerCoordination, RegionServerServices rsServices) {
@@ -214,7 +212,6 @@
    * which uses this method to do log splitting.
    * @return List of output files created by the split.
    */
-  @VisibleForTesting
   public static List<Path> split(Path walRootDir, Path walsDir, Path archiveDir, FileSystem walFS,
       Configuration conf, final WALFactory factory) throws IOException {
     Path rootDir = CommonFSUtils.getRootDir(conf);
@@ -287,7 +284,6 @@
    * WAL splitting implementation, splits one WAL file.
    * @param walStatus should be for an actual WAL file.
    */
-  @VisibleForTesting
   SplitWALResult splitWAL(FileStatus walStatus, CancelableProgressable cancel) throws IOException {
     Path wal = walStatus.getPath();
     Preconditions.checkArgument(walStatus.isFile(), "Not a regular file " + wal.toString());
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java
index a7bb8e6..f610ad6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java
@@ -43,8 +43,6 @@
 import org.apache.hadoop.hbase.nio.ByteBuff;
 import org.apache.hadoop.hbase.util.ChecksumType;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 public class CacheTestUtils {
 
   private static final boolean includesMemstoreTS = true;
@@ -326,7 +324,6 @@
     return returnedBlocks;
   }
 
-  @VisibleForTesting
   public static class HFileBlockPair {
     BlockCacheKey blockName;
     HFileBlock block;
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
index 27a74c3..9ba4e54 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
@@ -131,7 +131,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.base.Joiner;
 import org.apache.hbase.thirdparty.com.google.common.base.Splitter;
 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
@@ -331,7 +330,7 @@
    * the thrift server, not null means the server is started, for test only
    * @return the tServer
    */
-  @VisibleForTesting
+  @InterfaceAudience.Private
   public TServer getTserver() {
     return tserver;
   }
@@ -340,7 +339,7 @@
    * the Jetty server, not null means the HTTP server is started, for test only
    * @return the http server
    */
-  @VisibleForTesting
+  @InterfaceAudience.Private
   public Server getHttpServer() {
     return httpServer;
   }
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java
index 9259ec6..552ceec 100644
--- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java
@@ -42,7 +42,6 @@
 import org.apache.zookeeper.server.persistence.FileTxnLog;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 /**
  * TODO: Most of the code in this class is ripped from ZooKeeper tests. Instead
@@ -105,7 +104,8 @@
    *
    * @return clientPortList the client port list
    */
-  @VisibleForTesting public List<Integer> getClientPortList() {
+  @InterfaceAudience.Private
+  public List<Integer> getClientPortList() {
     return clientPortList;
   }