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 68e6b435871..0550f9bc147 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.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.Iterables; /** @@ -88,7 +87,7 @@ public class BackupHFileCleaner extends BaseHFileCleanerDelegate implements Abor } } - @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 d49aef2c0c4..e1fb73abe74 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.apache.yetus.audience.InterfaceAudience; 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 @@ public class BackupManager implements Closeable { * (TESTs only) * @param conf configuration */ - @VisibleForTesting public static void decorateMasterConfiguration(Configuration conf) { if (!isBackupEnabled(conf)) { return; @@ -137,7 +134,6 @@ public class BackupManager implements Closeable { * 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 83e545cb3bc..021341427a8 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.apache.yetus.audience.InterfaceAudience; 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 abstract class TableBackupClient { 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 class TableBackupClient { */ 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 7ec2a22e783..c987b49862d 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.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * Read/write operations on hbase:meta region as well as assignment information stored * to hbase:meta. @@ -262,7 +260,6 @@ public final class MetaTableAccessor { * true and we'll leave out offlined regions from returned list * @return List of all user-space regions. */ - @VisibleForTesting public static List getAllRegions(Connection connection, boolean excludeOfflinedSplitParents) throws IOException { List> result; @@ -861,7 +858,6 @@ public final class MetaTableAccessor { * @param sn Server name * @param masterSystemTime wall clock time from master if passed in the open region RPC */ - @VisibleForTesting public static void updateRegionLocation(Connection connection, RegionInfo regionInfo, ServerName sn, long openSeqNum, long masterSystemTime) throws IOException { updateLocation(connection, regionInfo, sn, openSeqNum, masterSystemTime); 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 16337072aa7..3cf6cc03523 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.apache.yetus.audience.InterfaceAudience; 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 @@ public class CatalogFamilyFormat { } /** The delimiter for meta columns for replicaIds > 0 */ - @VisibleForTesting static final char META_REPLICA_ID_DELIMITER = '_'; /** @@ -285,7 +282,6 @@ public class CatalogFamilyFormat { * @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 7aa9597c594..fcd1724d10e 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.conf.Configuration; import org.apache.hadoop.hbase.TableName; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer; import org.apache.hbase.thirdparty.io.netty.util.Timeout; @@ -60,7 +59,6 @@ class AsyncBufferedMutatorImpl implements AsyncBufferedMutator { private boolean closed; - @VisibleForTesting Timeout periodicFlushTask; AsyncBufferedMutatorImpl(HashedWheelTimer periodicalFlushTimer, AsyncTable table, @@ -83,7 +81,6 @@ class AsyncBufferedMutatorImpl implements AsyncBufferedMutator { } // will be overridden in test - @VisibleForTesting protected void internalFlush() { if (periodicFlushTask != null) { periodicFlushTask.cancel(); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java index fda262cfca3..51b84c09678 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.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService; @@ -75,7 +74,6 @@ class AsyncConnectionImpl implements AsyncConnection { private static final Logger LOG = LoggerFactory.getLogger(AsyncConnectionImpl.class); - @VisibleForTesting static final HashedWheelTimer RETRY_TIMER = new HashedWheelTimer( new ThreadFactoryBuilder().setNameFormat("Async-Client-Retry-Timer-pool-%d").setDaemon(true) .setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build(), 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 3571f96027f..5ae9de6c476 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.RegionLocations; 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 @@ class AsyncMetaRegionLocator { } // 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 a9ee6a9e552..2c2520f8bd1 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.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.base.Objects; /** @@ -74,13 +73,11 @@ class AsyncNonMetaRegionLocator { private static final Logger LOG = LoggerFactory.getLogger(AsyncNonMetaRegionLocator.class); - @VisibleForTesting static final String MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE = "hbase.client.meta.max.concurrent.locate.per.table"; private static final int DEFAULT_MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE = 8; - @VisibleForTesting static String LOCATE_PREFETCH_LIMIT = "hbase.client.locate.prefetch.limit"; private static final int DEFAULT_LOCATE_PREFETCH_LIMIT = 10; @@ -709,7 +706,6 @@ class AsyncNonMetaRegionLocator { } // only used for testing whether we have cached the location for a region. - @VisibleForTesting RegionLocations getRegionLocationInCache(TableName tableName, byte[] row) { TableCache tableCache = cache.get(tableName); if (tableCache == null) { @@ -719,7 +715,6 @@ class AsyncNonMetaRegionLocator { } // 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 09eabfc1d53..215a1c58faa 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.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer; import org.apache.hbase.thirdparty.io.netty.util.Timeout; @@ -175,13 +174,11 @@ class AsyncRegionLocator { 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 @@ class AsyncRegionLocator { } // 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 7fe6d120c3f..2858d2f915f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableResultScanner.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableResultScanner.java @@ -29,8 +29,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * The {@link ResultScanner} implementation for {@link AsyncTable}. It will fetch data automatically * in background and cache it in memory. Typically the {@link #maxCacheSize} will be @@ -177,7 +175,6 @@ class AsyncTableResultScanner implements ResultScanner, AdvancedScanResultConsum } // used in tests to test whether the scanner has been suspended - @VisibleForTesting synchronized boolean isSuspended() { return resumer != null; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java index 9edf8c251ee..19ca9adbf3f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java @@ -15,8 +15,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * Configuration parameters for the connection. * Configuration is a heavy weight registry that does a lot of string operations and regex matching. @@ -125,7 +123,6 @@ public class ConnectionConfiguration { * This is for internal testing purpose (using the default value). * In real usage, we should read the configuration from the Configuration object. */ - @VisibleForTesting protected ConnectionConfiguration() { this.writeBufferSize = WRITE_BUFFER_SIZE_DEFAULT; this.writeBufferPeriodicFlushTimeoutMs = WRITE_BUFFER_PERIODIC_FLUSH_TIMEOUT_MS_DEFAULT; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java index 06582684c79..0975289d516 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.security.User; import org.apache.hadoop.hbase.util.DNS.ServerType; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; import org.apache.hbase.thirdparty.com.google.common.base.Strings; import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap; @@ -151,7 +150,6 @@ public class MasterRegistry implements ConnectionRegistry { *

* Will be called in {@code HBaseTestingUtility}. */ - @VisibleForTesting public static String getMasterAddr(Configuration conf) throws UnknownHostException { String masterAddrFromConf = conf.get(MASTER_ADDRS_KEY); if (!Strings.isNullOrEmpty(masterAddrFromConf)) { @@ -331,7 +329,6 @@ public class MasterRegistry implements ConnectionRegistry { "getMasters()").thenApply(MasterRegistry::transformServerNames); } - @VisibleForTesting Set getParsedMasterServers() { return masterAddr2Stub.keySet(); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java index e9f4c61f5a2..f2e4218953f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java @@ -26,7 +26,6 @@ import com.codahale.metrics.JmxReporter; import com.codahale.metrics.MetricRegistry; import com.codahale.metrics.RatioGauge; import com.codahale.metrics.Timer; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -126,12 +125,11 @@ public class MetricsConnection implements StatisticTrackable { } } - @VisibleForTesting protected static final class CallTracker { private final String name; - @VisibleForTesting final Timer callTimer; - @VisibleForTesting final Histogram reqHist; - @VisibleForTesting final Histogram respHist; + final Timer callTimer; + final Histogram reqHist; + final Histogram respHist; private CallTracker(MetricRegistry registry, String name, String subName, String scope) { StringBuilder sb = new StringBuilder(CLIENT_SVC).append("_").append(name); @@ -182,7 +180,6 @@ public class MetricsConnection implements StatisticTrackable { } } - @VisibleForTesting protected static class RunnerStats { final Counter normalRunners; final Counter delayRunners; @@ -210,7 +207,6 @@ public class MetricsConnection implements StatisticTrackable { } } - @VisibleForTesting protected ConcurrentHashMap> serverStats = new ConcurrentHashMap<>(); @@ -275,36 +271,36 @@ public class MetricsConnection implements StatisticTrackable { // static metrics - @VisibleForTesting protected final Counter metaCacheHits; - @VisibleForTesting protected final Counter metaCacheMisses; - @VisibleForTesting protected final CallTracker getTracker; - @VisibleForTesting protected final CallTracker scanTracker; - @VisibleForTesting protected final CallTracker appendTracker; - @VisibleForTesting protected final CallTracker deleteTracker; - @VisibleForTesting protected final CallTracker incrementTracker; - @VisibleForTesting protected final CallTracker putTracker; - @VisibleForTesting protected final CallTracker multiTracker; - @VisibleForTesting protected final RunnerStats runnerStats; - @VisibleForTesting protected final Counter metaCacheNumClearServer; - @VisibleForTesting protected final Counter metaCacheNumClearRegion; - @VisibleForTesting protected final Counter hedgedReadOps; - @VisibleForTesting protected final Counter hedgedReadWin; - @VisibleForTesting protected final Histogram concurrentCallsPerServerHist; - @VisibleForTesting protected final Histogram numActionsPerServerHist; + protected final Counter metaCacheHits; + protected final Counter metaCacheMisses; + protected final CallTracker getTracker; + protected final CallTracker scanTracker; + protected final CallTracker appendTracker; + protected final CallTracker deleteTracker; + protected final CallTracker incrementTracker; + protected final CallTracker putTracker; + protected final CallTracker multiTracker; + protected final RunnerStats runnerStats; + protected final Counter metaCacheNumClearServer; + protected final Counter metaCacheNumClearRegion; + protected final Counter hedgedReadOps; + protected final Counter hedgedReadWin; + protected final Histogram concurrentCallsPerServerHist; + protected final Histogram numActionsPerServerHist; // dynamic metrics // These maps are used to cache references to the metric instances that are managed by the // registry. I don't think their use perfectly removes redundant allocations, but it's // a big improvement over calling registry.newMetric each time. - @VisibleForTesting protected final ConcurrentMap rpcTimers = + protected final ConcurrentMap rpcTimers = new ConcurrentHashMap<>(CAPACITY, LOAD_FACTOR, CONCURRENCY_LEVEL); - @VisibleForTesting protected final ConcurrentMap rpcHistograms = + protected final ConcurrentMap rpcHistograms = new ConcurrentHashMap<>(CAPACITY * 2 /* tracking both request and response sizes */, LOAD_FACTOR, CONCURRENCY_LEVEL); private final ConcurrentMap cacheDroppingExceptions = new ConcurrentHashMap<>(CAPACITY, LOAD_FACTOR, CONCURRENCY_LEVEL); - @VisibleForTesting protected final ConcurrentMap rpcCounters = + protected final ConcurrentMap rpcCounters = new ConcurrentHashMap<>(CAPACITY, LOAD_FACTOR, CONCURRENCY_LEVEL); MetricsConnection(String scope, Supplier batchPool, @@ -358,17 +354,14 @@ public class MetricsConnection implements StatisticTrackable { this.reporter.start(); } - @VisibleForTesting final String getExecutorPoolName() { return name(getClass(), "executorPoolActiveThreads", scope); } - @VisibleForTesting final String getMetaPoolName() { return name(getClass(), "metaPoolActiveThreads", scope); } - @VisibleForTesting MetricRegistry getMetricRegistry() { return registry; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java index 62879521afd..512e7a96aa6 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.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.Message; import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback; @@ -2385,7 +2384,6 @@ class RawAsyncHBaseAdmin implements AsyncAdmin { * @param regionNameOrEncodedRegionName region name or encoded region name * @return region location, wrapped by a {@link CompletableFuture} */ - @VisibleForTesting CompletableFuture getRegionLocation(byte[] regionNameOrEncodedRegionName) { if (regionNameOrEncodedRegionName == null) { return failedFuture(new IllegalArgumentException("Passed region name can't be null")); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SimpleRequestController.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SimpleRequestController.java index d87014428c3..1a184da86e4 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 @@ class SimpleRequestController implements RequestController { /** * Default value of {@link #HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE}. */ - @VisibleForTesting static final long DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE = 4194304; /** @@ -73,7 +71,6 @@ class SimpleRequestController implements RequestController { /** * Default value of {@link #HBASE_CLIENT_MAX_PERREQUEST_ROWS}. */ - @VisibleForTesting static final long DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_ROWS = 2048; /** @@ -83,14 +80,10 @@ class SimpleRequestController implements RequestController { /** * Default value of {@link #HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE}. */ - @VisibleForTesting static final long DEFAULT_HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE = DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE; - @VisibleForTesting final AtomicLong tasksInProgress = new AtomicLong(0); - @VisibleForTesting final ConcurrentMap taskCounterPerRegion = new ConcurrentSkipListMap<>(Bytes.BYTES_COMPARATOR); - @VisibleForTesting final ConcurrentMap taskCounterPerServer = new ConcurrentHashMap<>(); /** * The number of tasks simultaneously executed on the cluster. @@ -112,13 +105,11 @@ class SimpleRequestController implements RequestController { * don't start a set of operations on a region before the previous one is * done. As well, this limits the pressure we put on the region server. */ - @VisibleForTesting final int maxConcurrentTasksPerRegion; /** * The number of task simultaneously executed on a single region server. */ - @VisibleForTesting final int maxConcurrentTasksPerServer; private final int thresholdToLogUndoneTaskDetails; public static final String THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS = @@ -171,7 +162,6 @@ class SimpleRequestController implements RequestController { return value; } - @VisibleForTesting static Checker newChecker(List checkers) { return new Checker() { private boolean isEnd = false; @@ -331,7 +321,6 @@ class SimpleRequestController implements RequestController { * limit the heapsize of total submitted data. Reduce the limit of heapsize * for submitting quickly if there is no running task. */ - @VisibleForTesting static class SubmittedSizeChecker implements RowChecker { private final long maxHeapSizeSubmit; @@ -365,7 +354,6 @@ class SimpleRequestController implements RequestController { /** * limit the max number of tasks in an AsyncProcess. */ - @VisibleForTesting static class TaskCountChecker implements RowChecker { private static final long MAX_WAITING_TIME = 1000; //ms @@ -475,7 +463,6 @@ class SimpleRequestController implements RequestController { /** * limit the number of rows for each request. */ - @VisibleForTesting static class RequestRowsChecker implements RowChecker { private final long maxRowsPerRequest; @@ -514,7 +501,6 @@ class SimpleRequestController implements RequestController { /** * limit the heap size for each request. */ - @VisibleForTesting static class RequestHeapSizeChecker implements RowChecker { private final long maxHeapSizePerRequest; @@ -554,7 +540,6 @@ class SimpleRequestController implements RequestController { /** * Provide a way to control the flow of rows iteration. */ - @VisibleForTesting interface RowChecker { ReturnCode canTakeOperation(HRegionLocation loc, long heapSizeOfRow); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java index f1f05213863..4b31c7a6c8a 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java @@ -24,6 +24,7 @@ import static org.apache.hadoop.hbase.client.RegionReplicaUtil.getRegionInfoForR import static org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.lengthOfPBMagic; import static org.apache.hadoop.hbase.util.FutureUtils.addListener; import static org.apache.hadoop.hbase.zookeeper.ZKMetadata.removeMetaData; + import java.io.IOException; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -42,7 +43,7 @@ import org.apache.hadoop.hbase.zookeeper.ZNodePaths; 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 @@ class ZKConnectionRegistry implements ConnectionRegistry { return getAndConvert(znodePaths.clusterIdZNode, ZKConnectionRegistry::getClusterId); } - @VisibleForTesting ReadOnlyZKClient getZKClient() { return zk; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ClientExceptionsUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ClientExceptionsUtil.java index 6b1e251953b..2482a632ca8 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ClientExceptionsUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ClientExceptionsUtil.java @@ -28,7 +28,6 @@ import java.net.SocketTimeoutException; import java.nio.channels.ClosedChannelException; import java.util.Set; import java.util.concurrent.TimeoutException; - import org.apache.hadoop.hbase.CallDroppedException; import org.apache.hadoop.hbase.CallQueueTooBigException; import org.apache.hadoop.hbase.DoNotRetryIOException; @@ -36,16 +35,14 @@ import org.apache.hadoop.hbase.MultiActionResultTooLarge; import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.RegionTooBusyException; import org.apache.hadoop.hbase.RetryImmediatelyException; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.yetus.audience.InterfaceStability; - -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; -import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet; - import org.apache.hadoop.hbase.ipc.CallTimeoutException; import org.apache.hadoop.hbase.ipc.FailedServerException; import org.apache.hadoop.hbase.quotas.RpcThrottlingException; import org.apache.hadoop.ipc.RemoteException; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.yetus.audience.InterfaceStability; + +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet; @InterfaceAudience.Private @InterfaceStability.Evolving @@ -150,7 +147,6 @@ public final class ClientExceptionsUtil { * For test only. Usually you should use the {@link #isConnectionException(Throwable)} method * below. */ - @VisibleForTesting public static Set> getConnectionExceptionTypes() { return CONNECTION_EXCEPTION_TYPES; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java index f6811f607bd..9f44fe85bcc 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java @@ -37,8 +37,6 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.UnsafeAvailChecker; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * This is optimized version of a standard FuzzyRowFilter Filters data based on fuzzy row key. * Performs fast-forwards during scanning. It takes pairs (row key, fuzzy info) to match row keys. @@ -317,12 +315,10 @@ public class FuzzyRowFilter extends FilterBase { 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 @@ public class FuzzyRowFilter extends FilterBase { 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 @@ public class FuzzyRowFilter extends FilterBase { * @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 acc82de2942..be125f38377 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.PoolMap; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.ipc.RemoteException; -import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.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 @@ public abstract class AbstractRpcClient implements RpcC } } - @VisibleForTesting public static String getDefaultCodec(final Configuration c) { // If "hbase.client.default.rpc.codec" is empty string -- you can't set it to null because // Configuration will complain -- then no default codec (and we'll pb everything). Else @@ -248,7 +246,6 @@ public abstract class AbstractRpcClient implements RpcC } // for writing tests that want to throw exception when connecting. - @VisibleForTesting boolean isTcpNoDelay() { return tcpNoDelay; } @@ -565,7 +562,6 @@ public abstract class AbstractRpcClient implements RpcC /** * Blocking rpc channel that goes via hbase rpc. */ - @VisibleForTesting public static class BlockingRpcChannelImplementation extends AbstractRpcChannel implements BlockingRpcChannel { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java index 22eca535e95..dd8f96bb2b9 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java @@ -25,7 +25,6 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.client.MetricsConnection; import org.apache.hadoop.net.NetUtils; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; /** * Does RPC against a cluster. Manages connections per regionserver in the cluster. @@ -41,7 +40,6 @@ public class BlockingRpcClient extends AbstractRpcClient * Used in test only. Construct an IPC client for the cluster {@code clusterId} with the default * SocketFactory */ - @VisibleForTesting BlockingRpcClient(Configuration conf) { this(conf, HConstants.CLUSTER_ID_DEFAULT, null, null); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java index 811bb2c21fd..1b1411c7b95 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java @@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.ipc.RemoteException; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; import org.apache.hbase.thirdparty.com.google.protobuf.CodedOutputStream; import org.apache.hbase.thirdparty.com.google.protobuf.Message; @@ -248,7 +247,6 @@ class IPCUtil { } }; - @VisibleForTesting static final int MAX_DEPTH = 4; static void execute(EventLoop eventLoop, Runnable action) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientFactory.java index b6f9e385930..434795248c6 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientFactory.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientFactory.java @@ -17,15 +17,13 @@ */ package org.apache.hadoop.hbase.ipc; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; -import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap; - import java.net.SocketAddress; - import org.apache.hadoop.conf.Configuration; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.MetricsConnection; import org.apache.hadoop.hbase.util.ReflectionUtils; +import org.apache.yetus.audience.InterfaceAudience; + +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap; /** * Factory to create a {@link org.apache.hadoop.hbase.ipc.RpcClient} @@ -46,7 +44,6 @@ public final class RpcClientFactory { } /** Helper method for tests only. Creates an {@code RpcClient} without metrics. */ - @VisibleForTesting public static RpcClient createClient(Configuration conf, String clusterId) { return createClient(conf, clusterId, null); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java index 50594c81e5c..2d03473b9f6 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java @@ -23,7 +23,6 @@ import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos; @@ -188,7 +187,6 @@ public class RegionState { // The duration of region in transition private long ritDuration; - @VisibleForTesting public static RegionState createForTesting(RegionInfo region, State state) { return new RegionState(region, state, System.currentTimeMillis(), null); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/ThrottleSettings.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/ThrottleSettings.java index 3c25d6e4f94..d77d8d168c8 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/ThrottleSettings.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/ThrottleSettings.java @@ -19,12 +19,11 @@ package org.apache.hadoop.hbase.quotas; import java.io.IOException; import java.util.concurrent.TimeUnit; - import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.TableName; import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos; @@ -52,7 +51,6 @@ public class ThrottleSettings extends QuotaSettings { /** * Returns a copy of the internal state of this */ - @VisibleForTesting QuotaProtos.ThrottleRequest getProto() { return proto.toBuilder().build(); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java index f425984a95f..e00380d7171 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.hbase.util.VersionInfo; import org.apache.hadoop.ipc.RemoteException; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.io.ByteStreams; import org.apache.hbase.thirdparty.com.google.gson.JsonArray; import org.apache.hbase.thirdparty.com.google.gson.JsonElement; @@ -290,7 +289,6 @@ public final class ProtobufUtil { } } - @VisibleForTesting public static boolean isClassLoaderLoaded() { return classLoaderLoaded; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java index b4f3ccba9cd..a1475de8a1e 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java @@ -42,8 +42,6 @@ import org.apache.zookeeper.data.Stat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * A very simple read only zookeeper implementation without watcher support. */ @@ -117,7 +115,6 @@ public final class ReadOnlyZKClient implements Closeable { private final AtomicBoolean closed = new AtomicBoolean(false); - @VisibleForTesting ZooKeeper zookeeper; private int pendingRequests = 0; @@ -365,7 +362,6 @@ public final class ReadOnlyZKClient implements Closeable { } } - @VisibleForTesting public String getConnectString() { return connectString; } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java index caf11c3253f..a2a8198bd4d 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java @@ -25,8 +25,6 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * This Cell is an implementation of {@link ByteBufferExtendedCell} where the data resides in * off heap/ on heap ByteBuffer @@ -55,12 +53,10 @@ public class ByteBufferKeyValue extends ByteBufferExtendedCell { this.length = length; } - @VisibleForTesting public ByteBuffer getBuffer() { return this.buf; } - @VisibleForTesting public int getOffset() { return this.offset; } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java index 9dbb307df40..39c3ccc6919 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java @@ -32,8 +32,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * ChoreService is a service that can be used to schedule instances of {@link ScheduledChore} to run * periodically while sharing threads. The ChoreService is backed by a @@ -95,7 +93,6 @@ public class ChoreService implements ChoreServicer { * spawned by this service */ @InterfaceAudience.Private - @VisibleForTesting public ChoreService(final String coreThreadPoolPrefix) { this(coreThreadPoolPrefix, MIN_CORE_POOL_SIZE, false); } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java index b7ddbab6c90..856480f1506 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java @@ -40,8 +40,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * An HBase Key/Value. This is the fundamental HBase Type. *

@@ -1232,7 +1230,6 @@ public class KeyValue implements ExtendedCell, Cloneable { * and that we need access to the backing array to do some test case related assertions. * @return The byte array backing this KeyValue. */ - @VisibleForTesting public byte [] getBuffer() { return this.bytes; } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java index 2aadc4257c7..a86ca105912 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java @@ -40,8 +40,6 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * Utility methods helpful slinging {@link Cell} instances. It has more powerful and * rich set of APIs than those in {@link CellUtil} for internal usage. @@ -2635,7 +2633,6 @@ public final class PrivateCellUtil { * @return an int greater than 0 if left is greater than right lesser than 0 if left is lesser * than right equal to 0 if left is equal to right */ - @VisibleForTesting public static final int compare(CellComparator comparator, Cell left, byte[] key, int offset, int length) { // row diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ScheduledChore.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ScheduledChore.java index a009cf4ef12..1fb5b7e9e34 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ScheduledChore.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ScheduledChore.java @@ -24,8 +24,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * ScheduledChore is a task performed on a period in hbase. ScheduledChores become active once * scheduled with a {@link ChoreService} via {@link ChoreService#scheduleChore(ScheduledChore)}. The @@ -116,7 +114,6 @@ public abstract class ScheduledChore implements Runnable { * This constructor is for test only. It allows us to create an object and to call chore() on it. */ @InterfaceAudience.Private - @VisibleForTesting protected ScheduledChore() { this("TestChore", null, 0, DEFAULT_INITIAL_DELAY, DEFAULT_TIME_UNIT); } @@ -313,17 +310,17 @@ public abstract class ScheduledChore implements Runnable { return initialChoreComplete; } - @VisibleForTesting + @InterfaceAudience.Private synchronized ChoreServicer getChoreServicer() { return choreServicer; } - @VisibleForTesting + @InterfaceAudience.Private synchronized long getTimeOfLastRun() { return timeOfLastRun; } - @VisibleForTesting + @InterfaceAudience.Private synchronized long getTimeOfThisRun() { return timeOfThisRun; } @@ -336,7 +333,6 @@ public abstract class ScheduledChore implements Runnable { } @InterfaceAudience.Private - @VisibleForTesting public synchronized void choreForTesting() { chore(); } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/conf/ConfigurationManager.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/conf/ConfigurationManager.java index 2c36c5308fa..ad26f7633cb 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/conf/ConfigurationManager.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/conf/ConfigurationManager.java @@ -25,7 +25,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; /** * Maintains the set of all the classes which would like to get notified @@ -140,7 +139,6 @@ public class ConfigurationManager { /** * @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 8741bcff912..f0cb9b0a7d4 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.Queue; 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.hadoop.hbase.nio.SingleByteBuff; 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 @@ public class ByteBuffAllocator { return new ByteBuffAllocator(false, 0, DEFAULT_BUFFER_SIZE, Integer.MAX_VALUE); } - @VisibleForTesting ByteBuffAllocator(boolean reservoirEnabled, int maxBufCount, int bufSize, int minSizeForReservoirUse) { this.reservoirEnabled = reservoirEnabled; @@ -241,7 +237,6 @@ public class ByteBuffAllocator { * The {@link ConcurrentLinkedQueue#size()} is O(N) complexity and time-consuming, so DO NOT use * the method except in UT. */ - @VisibleForTesting public int getFreeBufferCount() { return this.buffers.size(); } @@ -348,7 +343,6 @@ public class ByteBuffAllocator { /** * Free all direct buffers if allocated, mainly used for testing. */ - @VisibleForTesting public void clean() { while (!buffers.isEmpty()) { ByteBuffer b = buffers.poll(); diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/aes/AES.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/aes/AES.java index 73b87135083..69f12f9ae06 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/aes/AES.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/aes/AES.java @@ -23,9 +23,7 @@ import java.io.OutputStream; import java.security.GeneralSecurityException; import java.security.Key; import java.security.SecureRandom; - import javax.crypto.spec.SecretKeySpec; - import org.apache.hadoop.hbase.io.crypto.Cipher; import org.apache.hadoop.hbase.io.crypto.CipherProvider; import org.apache.hadoop.hbase.io.crypto.Context; @@ -36,7 +34,6 @@ import org.apache.yetus.audience.InterfaceStability; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; /** @@ -149,7 +146,6 @@ public class AES extends Cipher { return d.createDecryptionStream(in); } - @VisibleForTesting SecureRandom getRNG() { return rng; } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/aes/CommonsCryptoAES.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/aes/CommonsCryptoAES.java index a119c57f59d..19c2bd8ae06 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/aes/CommonsCryptoAES.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/aes/CommonsCryptoAES.java @@ -24,9 +24,7 @@ import java.security.GeneralSecurityException; import java.security.Key; import java.security.SecureRandom; import java.util.Properties; - import javax.crypto.spec.SecretKeySpec; - import org.apache.commons.crypto.cipher.CryptoCipherFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.io.crypto.Cipher; @@ -39,7 +37,6 @@ import org.apache.yetus.audience.InterfaceStability; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; @InterfaceAudience.Private @@ -159,7 +156,6 @@ public class CommonsCryptoAES extends Cipher { return decryptor.createDecryptionStream(in); } - @VisibleForTesting SecureRandom getRNG() { return rng; } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java index afaf1976dc6..0f19254e34a 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java @@ -26,7 +26,6 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Iterator; import java.util.List; - import org.apache.commons.lang3.NotImplementedException; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HConstants; @@ -39,7 +38,6 @@ import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.compress.Compressor; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; /** @@ -48,7 +46,6 @@ import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; * This is used only in testing. */ @InterfaceAudience.Private -@VisibleForTesting public class EncodedDataBlock { private byte[] rawKVs; private ByteBuffer rawBuffer; diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hadoopbackport/ThrottledInputStream.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hadoopbackport/ThrottledInputStream.java index a488185591c..68627c3b1c8 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hadoopbackport/ThrottledInputStream.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hadoopbackport/ThrottledInputStream.java @@ -27,8 +27,6 @@ import org.apache.hadoop.fs.PositionedReadable; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * The ThrottleInputStream provides bandwidth throttling on a specified * InputStream. It is implemented as a wrapper on top of another InputStream @@ -126,7 +124,6 @@ public class ThrottledInputStream extends InputStream { EnvironmentEdgeManager.currentTime() - startTime); } - @VisibleForTesting static long calSleepTimeMs(long bytesRead, long maxBytesPerSec, long elapsed) { assert elapsed > 0 : "The elapsed time should be greater than zero"; if (bytesRead <= 0 || maxBytesPerSec <= 0) { diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/security/UserProvider.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/UserProvider.java index efa18fb9f58..0c054ceaaa2 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/security/UserProvider.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/UserProvider.java @@ -23,7 +23,6 @@ import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.hbase.BaseConfigurable; @@ -32,7 +31,6 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.ReflectionUtils; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder; import org.apache.hbase.thirdparty.com.google.common.cache.CacheLoader; import org.apache.hbase.thirdparty.com.google.common.cache.LoadingCache; @@ -57,7 +55,6 @@ public class UserProvider extends BaseConfigurable { static Groups groups = Groups.getUserToGroupsMappingService(); - @VisibleForTesting public static Groups getGroups() { return groups; } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java index 42d1bf4fa00..0c4c52f99fe 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java @@ -34,8 +34,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * This class manages an array of ByteBuffers with a default size 4MB. These buffers are sequential * and could be considered as a large buffer.It supports reading/writing data from this large buffer @@ -61,7 +59,6 @@ public class ByteBufferArray { Runtime.getRuntime().availableProcessors(), capacity, allocator); } - @VisibleForTesting ByteBufferArray(int bufferSize, int bufferCount, int threadCount, long capacity, ByteBufferAllocator alloc) throws IOException { this.bufferSize = bufferSize; @@ -107,7 +104,6 @@ public class ByteBufferArray { } } - @VisibleForTesting static int getBufferSize(long capacity) { int bufferSize = DEFAULT_BUFFER_SIZE; if (bufferSize > (capacity / 16)) { diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java index 08c35be0e0a..d270d63d635 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.BigDecimal; 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.hadoop.io.WritableUtils; 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 class ByteBufferUtils { public final static int VALUE_MASK = 0x7f; public final static int NEXT_BIT_SHIFT = 7; public final static int NEXT_BIT_MASK = 1 << 7; - @VisibleForTesting 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 6ecfa10d97b..ce24694e7fd 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.Collections; import java.util.Comparator; import java.util.Iterator; import java.util.List; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.io.RawComparator; @@ -46,10 +45,8 @@ import org.apache.hadoop.io.WritableUtils; 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 @@ public class Bytes implements Comparable { // SizeOf which uses java.lang.instrument says 24 bytes. (3 longs?) public static final int ESTIMATED_HEAP_TAX = 16; - @VisibleForTesting static final boolean UNSAFE_UNALIGNED = UnsafeAvailChecker.unaligned(); /** @@ -1270,7 +1266,6 @@ public class Bytes implements Comparable { } - @VisibleForTesting static Comparer lexicographicalComparerJavaImpl() { return LexicographicalComparerHolder.PureJavaComparer.INSTANCE; } @@ -1420,7 +1415,6 @@ public class Bytes implements Comparable { *

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 @@ public class Bytes implements Comparable { } } - @VisibleForTesting enum UnsafeComparer implements Comparer { INSTANCE; diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ClassSize.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ClassSize.java index 6f88c005cb8..b983fc0f3db 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 java.util.concurrent.ConcurrentSkipListMap; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; /** @@ -236,7 +235,6 @@ public class ClassSize { private static final MemoryLayout memoryLayout = getMemoryLayout(); private static final boolean USE_UNSAFE_LAYOUT = (memoryLayout instanceof UnsafeLayout); - @VisibleForTesting public static boolean useUnsafeLayout() { return USE_UNSAFE_LAYOUT; } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CommonFSUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CommonFSUtils.java index 5859c3ae73d..fef8e291b31 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CommonFSUtils.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CommonFSUtils.java @@ -28,7 +28,6 @@ import java.util.List; import java.util.Locale; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; -import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FSDataOutputStreamBuilder; @@ -41,12 +40,10 @@ import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.ipc.RemoteException; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; /** @@ -346,7 +343,7 @@ public final class CommonFSUtils { } /** - * Returns the URI in the strig format + * Returns the URI in the string format * @param c configuration * @param p path * @return - the URI's to string format @@ -359,7 +356,6 @@ public final class CommonFSUtils { return null; } - @VisibleForTesting public static void setWALRootDir(final Configuration c, final Path root) { c.set(HBASE_WAL_DIR, root.toString()); } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/IdLock.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/IdLock.java index 9e5692feebb..112af1ef85a 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/IdLock.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/IdLock.java @@ -21,12 +21,10 @@ import java.io.IOException; import java.io.InterruptedIOException; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; - import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; /** @@ -213,12 +211,10 @@ public class IdLock { } } - @VisibleForTesting void assertMapEmpty() { assert map.isEmpty(); } - @VisibleForTesting public void waitForWaiters(long id, int numWaiters) throws InterruptedException { for (Entry entry;;) { entry = map.get(id); diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java index ff2fd453835..f565bc1ad18 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java @@ -28,8 +28,6 @@ import java.nio.charset.Charset; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * Utility class that handles ordered byte arrays. That is, unlike * {@link Bytes}, these methods produce byte arrays which maintain the sort @@ -368,7 +366,6 @@ public class OrderedBytes { * @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 @@ public class OrderedBytes { * @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 @@ public class OrderedBytes { * @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 @@ public class OrderedBytes { * 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 @@ public class OrderedBytes { * 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 @@ public class OrderedBytes { /** * 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 98827772040..e4e4ce8e599 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.hbase.HConstants; import org.apache.hadoop.util.StringUtils; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * Utility methods for reading, and building the ZooKeeper configuration. * @@ -299,7 +297,6 @@ public final class ZKConfig { * @param clientPort the default client port * @return the string for a list of "server:port" separated by "," */ - @VisibleForTesting public static String standardizeZKQuorumServerString(String quorumStringInput, String clientPort) { String[] serverHosts = quorumStringInput.split(","); @@ -312,7 +309,6 @@ public final class ZKConfig { // in this case, the clientPort would be ignored) // (3). s1:p1,s2,s3:p3 (mix of (1) and (2) - if port is not specified in a server, it would use // the clientPort; otherwise, it would use the specified port) - @VisibleForTesting public static class ZKClusterKey { private String quorumString; private int clientPort; diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java index 16bce3e48b0..c30e32473b9 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java @@ -42,7 +42,7 @@ import org.junit.runners.Parameterized.Parameters; import org.junit.runners.model.Statement; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; import org.apache.hbase.thirdparty.com.google.common.collect.Sets; @@ -108,7 +108,6 @@ public final class HBaseClassTestRule implements TestRule { * @return the number of parameters for this given test class. If the test is not parameterized or * if there is any issue determining the number of parameters, returns 1. */ - @VisibleForTesting static int getNumParameters(Class clazz) { RunWith[] runWiths = clazz.getAnnotationsByType(RunWith.class); boolean testParameterized = runWiths != null && Arrays.stream(runWiths).anyMatch( diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/SimpleKdcServerUtil.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/SimpleKdcServerUtil.java index 32ffa7fcae2..79aae9b7879 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/SimpleKdcServerUtil.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/SimpleKdcServerUtil.java @@ -16,6 +16,7 @@ * limitations under the License. */ package org.apache.hadoop.hbase.util; + import java.io.File; import java.io.IOException; import java.net.BindException; @@ -26,7 +27,7 @@ import org.apache.kerby.kerberos.kerb.KrbException; import org.apache.kerby.kerberos.kerb.server.SimpleKdcServer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; /** @@ -58,7 +59,6 @@ public final class SimpleKdcServerUtil { * @return A running SimpleKdcServer on loopback/'localhost' on a random port * @see #getRunningSimpleKdcServer(File, Supplier) */ - @VisibleForTesting static SimpleKdcServer getRunningSimpleKdcServer(File testDir, Supplier randomPortGenerator, final boolean portClash) throws KrbException, IOException { diff --git a/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/coprocessor/Export.java b/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/coprocessor/Export.java index a93718f70db..2b38dcbaae4 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.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.protobuf.RpcCallback; import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; import org.apache.hbase.thirdparty.com.google.protobuf.Service; @@ -103,7 +102,7 @@ public class Export extends ExportProtos.ExportService implements RegionCoproces System.exit(response == null ? -1 : 0); } - @VisibleForTesting + @InterfaceAudience.Private static Map run(final Configuration conf, final String[] args) throws Throwable { String[] otherArgs = new GenericOptionsParser(conf, args).getRemainingArgs(); if (!ExportUtils.isValidArguements(args)) { diff --git a/hbase-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 96fc95416a8..a816d497044 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.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * This class acts as an adapter to export the MetricRegistry's in the global registry. Each * MetricRegistry will be registered or unregistered from the metric2 system. The collection will @@ -102,7 +100,6 @@ public final class GlobalMetricRegistriesAdapter { return new GlobalMetricRegistriesAdapter(); } - @VisibleForTesting public void stop() { stopped.set(true); } diff --git a/hbase-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 5e13a614ff0..dd143d4c6f5 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.MetricsRecordBuilder; import org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * Implementation of {@link MetricsTableLatencies} to track latencies for one table in a * RegionServer. @@ -36,7 +34,6 @@ public class MetricsTableLatenciesImpl extends BaseSourceImpl implements Metrics private final HashMap histogramsByTable = new HashMap<>(); - @VisibleForTesting public static class TableHistograms { final MetricHistogram getTimeHisto; final MetricHistogram incrementTimeHisto; @@ -120,7 +117,6 @@ public class MetricsTableLatenciesImpl extends BaseSourceImpl implements Metrics } } - @VisibleForTesting public static String qualifyMetricsName(TableName tableName, String metric) { StringBuilder sb = new StringBuilder(); sb.append("Namespace_").append(tableName.getNamespaceAsString()); @@ -129,7 +125,6 @@ public class MetricsTableLatenciesImpl extends BaseSourceImpl implements Metrics return sb.toString(); } - @VisibleForTesting public TableHistograms getOrCreateTableHistogram(String tableName) { // TODO Java8's ConcurrentHashMap#computeIfAbsent would be stellar instead final TableName tn = TableName.valueOf(tableName); diff --git a/hbase-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 fbcd9fc7eaf..88b491ba3ea 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.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * JMX caches the beans that have been exported; even after the values are removed from hadoop's * metrics system the keys and old values will still remain. This class stops and restarts the @@ -75,7 +73,6 @@ public final class JmxCacheBuster { * Stops the clearing of JMX metrics and restarting the Hadoop metrics system. This is needed for * some test environments where we manually inject sources or sinks dynamically. */ - @VisibleForTesting public static void stop() { stopped.set(true); ScheduledFuture future = fut.get(); @@ -86,7 +83,6 @@ public final class JmxCacheBuster { * Restarts the stopped service. * @see #stop() */ - @VisibleForTesting public static void restart() { stopped.set(false); } diff --git a/hbase-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 8d075589d4a..c1880f8203b 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 java.util.Map; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * Implementation of the Cormode, Korn, Muthukrishnan, and Srivastava algorithm * for streaming calculation of targeted high-percentile epsilon-approximate @@ -257,7 +255,6 @@ public class MetricSampleQuantiles { * * @return count current number of samples */ - @VisibleForTesting synchronized public int getSampleCount() { return samples.size(); } diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java index 1863557529d..41844e75543 100644 --- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java +++ b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java @@ -67,7 +67,6 @@ import org.apache.yetus.audience.InterfaceStability; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hbase.thirdparty.org.eclipse.jetty.http.HttpVersion; @@ -174,7 +173,6 @@ public class HttpServer implements FilterContainer { private final List listeners = Lists.newArrayList(); - @VisibleForTesting public List getServerConnectors() { return listeners.stream().map(info -> info.listener).collect(Collectors.toList()); } @@ -1122,7 +1120,6 @@ public class HttpServer implements FilterContainer { * Open the main listener for the server * @throws Exception if the listener cannot be opened or the appropriate port is already in use */ - @VisibleForTesting void openListeners() throws Exception { for (ListenerInfo li : listeners) { ServerConnector listener = li.listener; diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/log/LogLevel.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/log/LogLevel.java index dede1f9d8fd..1fcfa1390c2 100644 --- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/log/LogLevel.java +++ b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/log/LogLevel.java @@ -24,6 +24,7 @@ import java.io.InputStreamReader; import java.io.PrintWriter; import java.net.URL; import java.net.URLConnection; +import java.nio.charset.StandardCharsets; import java.util.Objects; import java.util.regex.Pattern; import javax.net.ssl.HttpsURLConnection; @@ -47,9 +48,6 @@ import org.apache.yetus.audience.InterfaceStability; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; -import org.apache.hbase.thirdparty.com.google.common.base.Charsets; - /** * Change log level in runtime. */ @@ -89,7 +87,6 @@ public final class LogLevel { protocol.equals(PROTOCOL_HTTPS))); } - @VisibleForTesting static class CLI extends Configured implements Tool { private Operations operation = Operations.UNKNOWN; private String protocol; @@ -289,7 +286,7 @@ public final class LogLevel { // read from the servlet try (InputStreamReader streamReader = - new InputStreamReader(connection.getInputStream(), Charsets.UTF_8); + new InputStreamReader(connection.getInputStream(), StandardCharsets.UTF_8); BufferedReader bufferedReader = new BufferedReader(streamReader)) { bufferedReader.lines().filter(Objects::nonNull).filter(line -> line.startsWith(MARKER)) .forEach(line -> System.out.println(TAG.matcher(line).replaceAll(""))); diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/Driver.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/Driver.java index da907cbff77..0484fbbf239 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.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; import org.apache.hadoop.util.ProgramDriver; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * Driver for hbase mapreduce jobs. Select which to run by passing name of job * to this main. @@ -35,7 +33,7 @@ public class Driver { private static ProgramDriver pgd = new ProgramDriver(); - @VisibleForTesting + @InterfaceAudience.Private static void setProgramDriver(ProgramDriver pgd0) { pgd = pgd0; } diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/ExportUtils.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/ExportUtils.java index a4f092b7179..568c47fd6e5 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.PrefixFilter; import org.apache.hadoop.hbase.filter.RegexStringComparator; import org.apache.hadoop.hbase.filter.RowFilter; import org.apache.hadoop.hbase.security.visibility.Authorizations; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Triple; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; @@ -106,7 +105,6 @@ public final class ExportUtils { return new Triple<>(TableName.valueOf(args[0]), getScanFromCommandLine(conf, args), new Path(args[1])); } - @VisibleForTesting static Scan getScanFromCommandLine(Configuration conf, String[] args) throws IOException { Scan s = new Scan(); // Optional arguments. diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java index 08752c192b6..75b5246d2c8 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java @@ -90,8 +90,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * Writes HFiles. Passed Cells must arrive in order. * Writes current time as the sequence id for the file. Sets the major compacted @@ -665,7 +663,7 @@ public class HFileOutputFormat2 * @param conf to read the serialized values from * @return a map from column family to the configured compression algorithm */ - @VisibleForTesting + @InterfaceAudience.Private static Map createFamilyCompressionMap(Configuration conf) { Map stringMap = createFamilyConfValueMap(conf, @@ -685,7 +683,7 @@ public class HFileOutputFormat2 * @param conf to read the serialized values from * @return a map from column family to the the configured bloom filter type */ - @VisibleForTesting + @InterfaceAudience.Private static Map createFamilyBloomTypeMap(Configuration conf) { Map stringMap = createFamilyConfValueMap(conf, BLOOM_TYPE_FAMILIES_CONF_KEY); @@ -704,12 +702,11 @@ public class HFileOutputFormat2 * @param conf to read the serialized values from * @return a map from column family to the the configured bloom filter param */ - @VisibleForTesting + @InterfaceAudience.Private static Map createFamilyBloomParamMap(Configuration conf) { return createFamilyConfValueMap(conf, BLOOM_PARAM_FAMILIES_CONF_KEY); } - /** * Runs inside the task to deserialize column family to block size * map from the configuration. @@ -717,7 +714,7 @@ public class HFileOutputFormat2 * @param conf to read the serialized values from * @return a map from column family to the configured block size */ - @VisibleForTesting + @InterfaceAudience.Private static Map createFamilyBlockSizeMap(Configuration conf) { Map stringMap = createFamilyConfValueMap(conf, BLOCK_SIZE_FAMILIES_CONF_KEY); @@ -737,7 +734,7 @@ public class HFileOutputFormat2 * @return a map from column family to HFileDataBlockEncoder for the * configured data block type for the family */ - @VisibleForTesting + @InterfaceAudience.Private static Map createFamilyDataBlockEncodingMap( Configuration conf) { Map stringMap = createFamilyConfValueMap(conf, @@ -749,7 +746,6 @@ public class HFileOutputFormat2 return encoderMap; } - /** * Run inside the task to deserialize column family to given conf value map. * @@ -802,7 +798,7 @@ public class HFileOutputFormat2 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE") - @VisibleForTesting + @InterfaceAudience.Private static String serializeColumnFamilyAttribute(Function fn, List allTables) throws UnsupportedEncodingException { @@ -833,7 +829,7 @@ public class HFileOutputFormat2 * Serialize column family to compression algorithm map to configuration. * Invoked while configuring the MR job for incremental load. */ - @VisibleForTesting + @InterfaceAudience.Private static Function compressionDetails = familyDescriptor -> familyDescriptor.getCompressionType().getName(); @@ -841,7 +837,7 @@ public class HFileOutputFormat2 * Serialize column family to block size map to configuration. Invoked while * configuring the MR job for incremental load. */ - @VisibleForTesting + @InterfaceAudience.Private static Function blockSizeDetails = familyDescriptor -> String .valueOf(familyDescriptor.getBlocksize()); @@ -849,7 +845,7 @@ public class HFileOutputFormat2 * Serialize column family to bloom type map to configuration. Invoked while * configuring the MR job for incremental load. */ - @VisibleForTesting + @InterfaceAudience.Private static Function bloomTypeDetails = familyDescriptor -> { String bloomType = familyDescriptor.getBloomFilterType().toString(); if (bloomType == null) { @@ -862,7 +858,7 @@ public class HFileOutputFormat2 * Serialize column family to bloom param map to configuration. Invoked while * configuring the MR job for incremental load. */ - @VisibleForTesting + @InterfaceAudience.Private static Function bloomParamDetails = familyDescriptor -> { BloomType bloomType = familyDescriptor.getBloomFilterType(); String bloomParam = ""; @@ -876,7 +872,7 @@ public class HFileOutputFormat2 * Serialize column family to data block encoding map to configuration. * Invoked while configuring the MR job for incremental load. */ - @VisibleForTesting + @InterfaceAudience.Private static Function dataBlockEncodingDetails = familyDescriptor -> { DataBlockEncoding encoding = familyDescriptor.getDataBlockEncoding(); if (encoding == null) { diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableHFileOutputFormat.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableHFileOutputFormat.java index cdc2f94c740..6410bf8726c 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableHFileOutputFormat.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableHFileOutputFormat.java @@ -29,7 +29,6 @@ import org.apache.hadoop.mapreduce.Job; import java.io.IOException; import java.nio.charset.Charset; import java.util.List; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; /** * Create 3 level tree directory, first level is using table name as parent @@ -45,7 +44,6 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti * -columnFamilyName2 */ @InterfaceAudience.Public -@VisibleForTesting public class MultiTableHFileOutputFormat extends HFileOutputFormat2 { private static final Logger LOG = LoggerFactory.getLogger(MultiTableHFileOutputFormat.class); diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java index 8baf85ffb4d..66790104223 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java @@ -51,7 +51,6 @@ import org.apache.hadoop.util.StringUtils; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; /** * A base for {@link TableInputFormat}s. Receives a {@link Connection}, a {@link TableName}, @@ -600,7 +599,7 @@ public abstract class TableInputFormatBase this.connection = connection; } - @VisibleForTesting + @InterfaceAudience.Private protected RegionSizeCalculator createRegionSizeCalculator(RegionLocator locator, Admin admin) throws IOException { return new RegionSizeCalculator(locator, admin); diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReaderImpl.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReaderImpl.java index c84b25b0004..81f9a7ceb43 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.hadoop.util.StringUtils; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; /** * Iterate over an HBase table data, return (ImmutableBytesWritable, Result) @@ -51,8 +50,9 @@ public class TableRecordReaderImpl { private static final Logger LOG = LoggerFactory.getLogger(TableRecordReaderImpl.class); // HBASE_COUNTER_GROUP_NAME is the name of mapreduce counter group for HBase - @VisibleForTesting + @InterfaceAudience.Private static final String HBASE_COUNTER_GROUP_NAME = "HBaseCounters"; + private ResultScanner scanner = null; private Scan scan = null; private Scan currentScan = null; diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormat.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormat.java index 3ca6c032368..d9c9b871a59 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.RecordReader; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * TableSnapshotInputFormat allows a MapReduce job to run over a table snapshot. The job * bypasses HBase servers, and directly accesses the underlying files (hfile, recovered edits, @@ -131,7 +129,7 @@ public class TableSnapshotInputFormat extends InputFormat { private TableSnapshotInputFormatImpl.RecordReader delegate = diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java index 7b940d724ba..baa0d1d0246 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java @@ -69,8 +69,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * This map-only job compares the data from a local table with a remote one. * Every cell is compared and must have exactly the same keys (even timestamp) @@ -517,7 +515,6 @@ public class VerifyReplication extends Configured implements Tool { scan.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 e17a28a5828..edc813d95b9 100644 --- a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistriesLoader.java +++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistriesLoader.java @@ -28,8 +28,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - @InterfaceAudience.Private public final class MetricRegistriesLoader { private static final Logger LOG = LoggerFactory.getLogger(MetricRegistries.class); @@ -57,7 +55,6 @@ public final class MetricRegistriesLoader { * implementation will be loaded. * @return A {@link MetricRegistries} implementation. */ - @VisibleForTesting static MetricRegistries load(List availableImplementations) { if (availableImplementations.size() == 1) { diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/AbstractProcedureScheduler.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/AbstractProcedureScheduler.java index 6b4461402f2..53bfba62daf 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.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - @InterfaceAudience.Private public abstract class AbstractProcedureScheduler implements ProcedureScheduler { private static final Logger LOG = LoggerFactory.getLogger(AbstractProcedureScheduler.class); @@ -246,7 +244,6 @@ public abstract class AbstractProcedureScheduler implements ProcedureScheduler { * 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 46dd9e289b1..579c6099876 100644 --- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java +++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java @@ -33,8 +33,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState; /** @@ -588,7 +586,6 @@ public abstract class Procedure implements Comparable implements Comparable implements Comparable { * 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 @@ public class ProcedureEvent { * 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 b99f544628b..9111345fe8d 100644 --- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java +++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java @@ -56,7 +56,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -639,7 +638,6 @@ public class ProcedureExecutor { workerMonitorExecutor.sendStopSignal(); } - @VisibleForTesting public void join() { assert !isRunning() : "expected not running"; @@ -1332,12 +1330,10 @@ public class ProcedureExecutor { return procId; } - @VisibleForTesting protected long getLastProcId() { return lastProcId.get(); } - @VisibleForTesting public Set getActiveProcIds() { return procedures.keySet(); } @@ -1932,17 +1928,14 @@ public class ProcedureExecutor { return rollbackStack.get(rootProcId); } - @VisibleForTesting ProcedureScheduler getProcedureScheduler() { return scheduler; } - @VisibleForTesting int getCompletedSize() { return completed.size(); } - @VisibleForTesting public IdLock getProcExecutionLock() { return procExecutionLock; } diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureScheduler.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureScheduler.java index d787cc0979c..72b2b284ca1 100644 --- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureScheduler.java +++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureScheduler.java @@ -22,8 +22,6 @@ import java.util.List; import java.util.concurrent.TimeUnit; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * Keep track of the runnable procedures */ @@ -125,7 +123,6 @@ public interface ProcedureScheduler { * Returns the number of elements in this queue. * @return the number of elements in this queue. */ - @VisibleForTesting int size(); /** @@ -133,6 +130,5 @@ public interface ProcedureScheduler { * Used for testing failure and recovery. To emulate server crash/restart, * {@link ProcedureExecutor} resets its own state and calls clear() on scheduler. */ - @VisibleForTesting void clear(); } diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SimpleProcedureScheduler.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SimpleProcedureScheduler.java index feab8be16c0..2b043d472d0 100644 --- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SimpleProcedureScheduler.java +++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SimpleProcedureScheduler.java @@ -23,8 +23,6 @@ import java.util.List; import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * Simple scheduler for procedures */ @@ -47,7 +45,6 @@ public class SimpleProcedureScheduler extends AbstractProcedureScheduler { return runnables.poll(); } - @VisibleForTesting @Override public void clear() { schedLock(); diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java index 46c4c5e545e..d1af4969141 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.ArrayList; 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 @@ public abstract class StateMachineProcedure */ private int previousState; - @VisibleForTesting public enum Flow { HAS_MORE_STATE, NO_MORE_STATE, @@ -282,7 +278,6 @@ public abstract class StateMachineProcedure * sequentially. Some procedures may skip steps/ states, some may add intermediate steps in * future. */ - @VisibleForTesting public int getCurrentStateId() { return getStateId(getCurrentState()); } diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java index b0301c67606..6aed228af5a 100644 --- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java +++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java @@ -59,7 +59,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.org.apache.commons.collections4.queue.CircularFifoQueue; import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALHeader; @@ -240,7 +239,6 @@ public class WALProcedureStore extends ProcedureStoreBase { leaseRecovery); } - @VisibleForTesting public WALProcedureStore(final Configuration conf, final Path walDir, final Path walArchiveDir, final LeaseRecovery leaseRecovery) throws IOException { this.conf = conf; @@ -984,7 +982,6 @@ public class WALProcedureStore extends ProcedureStoreBase { return (System.currentTimeMillis() - lastRollTs.get()); } - @VisibleForTesting void periodicRollForTesting() throws IOException { lock.lock(); try { @@ -994,7 +991,6 @@ public class WALProcedureStore extends ProcedureStoreBase { } } - @VisibleForTesting public boolean rollWriterForTesting() throws IOException { lock.lock(); try { @@ -1004,7 +1000,6 @@ public class WALProcedureStore extends ProcedureStoreBase { } } - @VisibleForTesting void removeInactiveLogsForTesting() throws Exception { lock.lock(); try { @@ -1058,7 +1053,6 @@ public class WALProcedureStore extends ProcedureStoreBase { return true; } - @VisibleForTesting boolean rollWriter(long logId) throws IOException { assert logId > flushLogId : "logId=" + logId + " flushLogId=" + flushLogId; assert lock.isHeldByCurrentThread() : "expected to be the lock owner. " + lock.isLocked(); @@ -1257,7 +1251,6 @@ public class WALProcedureStore extends ProcedureStoreBase { return this.walDir; } - @VisibleForTesting Path getWalArchiveDir() { return this.walArchiveDir; } diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java index ba6da7af3be..ebe99da3541 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java @@ -27,8 +27,6 @@ import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * This provides an class for maintaining a set of peer clusters. These peers are remote slave * clusters that data is replicated to. @@ -59,7 +57,6 @@ public class ReplicationPeers { } } - @VisibleForTesting public ReplicationPeerStorage getPeerStorage() { return this.peerStorage; } diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java index 7a943c4035c..09aeee55cca 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.hadoop.hbase.zookeeper.ZNodePaths; import org.apache.yetus.audience.InterfaceAudience; import org.apache.zookeeper.KeeperException; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos; /** @@ -78,22 +76,18 @@ public class ZKReplicationPeerStorage extends ZKReplicationStorageBase this.peersZNode = ZNodePaths.joinZNode(replicationZNode, peersZNodeName); } - @VisibleForTesting public String getPeerStateNode(String peerId) { return ZNodePaths.joinZNode(getPeerNode(peerId), peerStateNodeName); } - @VisibleForTesting public String getPeerNode(String peerId) { return ZNodePaths.joinZNode(peersZNode, peerId); } - @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 6f1f5a3d6a4..5c480bacdd8 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java @@ -51,7 +51,6 @@ import org.apache.zookeeper.data.Stat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils; /** @@ -103,7 +102,6 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase */ private final String hfileRefsZNode; - @VisibleForTesting final String regionsZNode; public ZKReplicationQueueStorage(ZKWatcher zookeeper, Configuration conf) { @@ -158,7 +156,6 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase * @return ZNode path to persist the max sequence id that we've pushed for the given region and * peer. */ - @VisibleForTesting String getSerialReplicationRegionPeerNode(String encodedRegionName, String peerId) { if (encodedRegionName == null || encodedRegionName.length() != RegionInfo.MD5_HEX_LENGTH) { throw new IllegalArgumentException( @@ -264,7 +261,6 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase * Return the {lastPushedSequenceId, ZNodeDataVersion} pair. if ZNodeDataVersion is -1, it means * that the ZNode does not exist. */ - @VisibleForTesting protected Pair getLastSequenceIdWithVersion(String encodedRegionName, String peerId) throws KeeperException { Stat stat = new Stat(); @@ -503,7 +499,6 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase } // will be overridden in UTs - @VisibleForTesting protected int getQueuesZNodeCversion() throws KeeperException { Stat stat = new Stat(); ZKUtil.getDataNoWatch(this.zookeeper, this.queuesZNode, stat); @@ -641,7 +636,6 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase } // will be overridden in UTs - @VisibleForTesting protected int getHFileRefsZNodeCversion() throws ReplicationException { Stat stat = new Stat(); try { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ExecutorStatusChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ExecutorStatusChore.java index da03eba0351..4b4aef30bbc 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.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; /** * The Class ExecutorStatusChore for collect Executor status info periodically @@ -77,7 +76,6 @@ public class ExecutorStatusChore extends ScheduledChore { } } - @VisibleForTesting public Pair 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 e49fe8c8c93..e27574a0f92 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.LockR 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 @@ public class EntityLock { return sb.toString(); } - @VisibleForTesting + @InterfaceAudience.Private void setTestingSleepTime(long timeInMillis) { testingSleepTime = timeInMillis; } - @VisibleForTesting + @InterfaceAudience.Private LockHeartbeatWorker getWorker() { return worker; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/locking/LockServiceClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/locking/LockServiceClient.java index 4229a7aa6fc..24f2835af8b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/locking/LockServiceClient.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/locking/LockServiceClient.java @@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.client.locking; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.HBaseInterfaceAudience; @@ -30,7 +29,6 @@ import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockService; @@ -89,7 +87,7 @@ public class LockServiceClient { return new EntityLock(conf, stub, lockRequest, abort); } - @VisibleForTesting + @InterfaceAudience.Private public static LockRequest buildLockRequest(final LockType type, final String namespace, final TableName tableName, final List regionInfos, final String description, final long nonceGroup, final long nonce) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitLogManagerCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitLogManagerCoordination.java index 33d8f2ca779..9508321a625 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitLogManagerCoordination.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitLogManagerCoordination.java @@ -29,8 +29,6 @@ import org.apache.hadoop.hbase.master.SplitLogManager.ResubmitDirective; import org.apache.hadoop.hbase.master.SplitLogManager.Task; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * Coordination for SplitLogManager. It creates and works with tasks for split log operations
* Manager prepares task by calling {@link #prepareTask} and submit it by @@ -147,6 +145,5 @@ public interface SplitLogManagerCoordination { * Support method to init constants such as timeout. Mostly required for UTs. * @throws IOException */ - @VisibleForTesting void init() throws IOException; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitLogWorkerCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitLogWorkerCoordination.java index a9fae4640d5..5452578a2c2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitLogWorkerCoordination.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitLogWorkerCoordination.java @@ -25,7 +25,6 @@ import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.SplitLogWorker; import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; /** * Coordinated operations for {@link SplitLogWorker} and @@ -94,7 +93,6 @@ public interface SplitLogWorkerCoordination { * Used by unit tests to check how many tasks were processed * @return number of tasks */ - @VisibleForTesting int getTaskReadySeq(); /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java index 8fc351fb775..dee94be9fad 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java @@ -58,7 +58,6 @@ import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.data.Stat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; /** * ZooKeeper based implementation of @@ -742,7 +741,6 @@ public class ZKSplitLogManagerCoordination extends ZKListener implements /** * Temporary function that is used by unit tests only */ - @VisibleForTesting public void setIgnoreDeleteForTesting(boolean b) { ignoreZKDeleteForTesting = b; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java index 7e0f2ae2b86..44250763005 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java @@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.security.User; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.hbase.util.CoprocessorClassLoader; import org.apache.hadoop.hbase.util.SortedList; @@ -253,7 +252,6 @@ public abstract class CoprocessorHost implClass, int priority, Configuration conf) throws IOException { E env = checkAndLoadInstance(implClass, priority, conf); @@ -325,7 +323,6 @@ public abstract class CoprocessorHost T findCoprocessor(Class cls) { for (E env: coprocEnvironments) { if (cls.isAssignableFrom(env.getInstance().getClass())) { @@ -360,7 +357,6 @@ public abstract class CoprocessorHost implements Ob * @return An instance of ObserverContext with the environment set */ @Deprecated - @VisibleForTesting // TODO: Remove this method, ObserverContext should not depend on RpcServer public static ObserverContext createAndPrepare(E env) { ObserverContextImpl ctx = new ObserverContextImpl<>(RpcServer.getRequestUser().orElse(null)); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java index c12c30aad67..8d2d7e2afb6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorService.java @@ -32,13 +32,11 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; - import org.apache.hadoop.hbase.monitoring.ThreadMonitoring; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hbase.thirdparty.com.google.common.collect.Maps; import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ListenableFuture; @@ -85,7 +83,6 @@ public class ExecutorService { * started with the same name, this throws a RuntimeException. * @param name Name of the service to start. */ - @VisibleForTesting public void startExecutorService(String name, int maxThreads) { Executor hbes = this.executorMap.compute(name, (key, value) -> { if (value != null) { @@ -126,7 +123,6 @@ public class ExecutorService { return executor; } - @VisibleForTesting public ThreadPoolExecutor getExecutorThreadPool(final ExecutorType type) { return getExecutor(type).getThreadPoolExecutor(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesManager.java index a34e3a3183a..16f0934a291 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesManager.java @@ -22,6 +22,7 @@ import static org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper.FAVORE import static org.apache.hadoop.hbase.favored.FavoredNodesPlan.Position.PRIMARY; import static org.apache.hadoop.hbase.favored.FavoredNodesPlan.Position.SECONDARY; import static org.apache.hadoop.hbase.favored.FavoredNodesPlan.Position.TERTIARY; + import java.io.IOException; import java.util.ArrayList; import java.util.Collection; @@ -42,7 +43,7 @@ import org.apache.hadoop.net.NetUtils; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hbase.thirdparty.com.google.common.collect.Maps; import org.apache.hbase.thirdparty.com.google.common.collect.Sets; @@ -95,7 +96,6 @@ public class FavoredNodesManager { datanodeDataTransferPort= getDataNodePort(); } - @VisibleForTesting public int getDataNodePort() { HdfsConfiguration.init(); @@ -282,7 +282,6 @@ public class FavoredNodesManager { } } - @VisibleForTesting public synchronized Set getRegionsOfFavoredNode(ServerName serverName) { Set regionInfos = Sets.newHashSet(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java index c1f9a7ddacc..d83a9d9da90 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.FSDataInputStream; 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 @@ public class FSDataInputStreamWrapper implements Closeable { } /** For use in tests. */ - @VisibleForTesting public FSDataInputStreamWrapper(FSDataInputStream fsdis) { this(fsdis, fsdis); } /** For use in tests. */ - @VisibleForTesting public FSDataInputStreamWrapper(FSDataInputStream fsdis, FSDataInputStream noChecksum) { doCloseStreams = false; stream = fsdis; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/MetricsIO.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/MetricsIO.java index 3634ccb595f..72da73e1e92 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/MetricsIO.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/MetricsIO.java @@ -22,8 +22,6 @@ import org.apache.hadoop.hbase.CompatibilitySingletonFactory; import org.apache.hadoop.hbase.regionserver.MetricsRegionServerSourceFactory; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - @InterfaceAudience.Private public class MetricsIO { @@ -40,12 +38,10 @@ public class MetricsIO { this.wrapper = wrapper; } - @VisibleForTesting public MetricsIOSource getMetricsSource() { return source; } - @VisibleForTesting public MetricsIOWrapper getWrapper() { return wrapper; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java index efc83af6a1b..dcbb71582f4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java @@ -27,8 +27,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * Stores all of the cache objects and configuration for a single HFile. */ @@ -283,7 +281,6 @@ public class CacheConfig { * @param cacheDataOnWrite whether data blocks should be written to the cache * when an HFile is written */ - @VisibleForTesting public void setCacheDataOnWrite(boolean cacheDataOnWrite) { this.cacheDataOnWrite = cacheDataOnWrite; } @@ -329,7 +326,6 @@ public class CacheConfig { * @param evictOnClose whether blocks should be evicted from the cache when an * HFile reader is closed */ - @VisibleForTesting public void setEvictOnClose(boolean evictOnClose) { this.evictOnClose = evictOnClose; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java index 5544ecef374..1fb87133f20 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java @@ -25,9 +25,6 @@ import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.io.hfile.BlockType.BlockCategory; import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - - /** * CombinedBlockCache is an abstraction layer that combines * {@link FirstLevelBlockCache} and {@link BucketCache}. The smaller lruCache is used @@ -397,7 +394,6 @@ public class CombinedBlockCache implements ResizableBlockCache, HeapSize { this.l1Cache.setMaxSize(size); } - @VisibleForTesting public int getRpcRefCount(BlockCacheKey cacheKey) { return (this.l2Cache instanceof BucketCache) ? ((BucketCache) this.l2Cache).getRpcRefCount(cacheKey) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java index cae284a1d40..7ab4edb438a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java @@ -204,7 +204,6 @@ public class FixedFileTrailer { baos.writeTo(outputStream); } - @org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting HFileProtos.FileTrailerProto toProtobuf() { HFileProtos.FileTrailerProto.Builder builder = HFileProtos.FileTrailerProto.newBuilder() .setFileInfoOffset(fileInfoOffset) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java index 46cec4af1f8..52b6359d92c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java @@ -51,7 +51,7 @@ import org.apache.hadoop.io.Writable; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; /** @@ -459,10 +459,8 @@ public final class HFile { DataBlockEncoding getEffectiveEncodingInCache(boolean isCompaction); - @VisibleForTesting HFileBlock.FSReader getUncachedBlockReader(); - @VisibleForTesting boolean prefetchComplete(); /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java index f4fdb9b2732..a02ad7d4e10 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java @@ -55,7 +55,7 @@ import org.apache.hadoop.hbase.util.ClassSize; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; /** @@ -311,7 +311,6 @@ public class HFileBlock implements Cacheable { * @param onDiskDataSizeWithHeader see {@link #onDiskDataSizeWithHeader} * @param fileContext HFile meta data */ - @VisibleForTesting public HFileBlock(BlockType blockType, int onDiskSizeWithoutHeader, int uncompressedSizeWithoutHeader, long prevBlockOffset, ByteBuff buf, boolean fillHeader, long offset, int nextBlockOnDiskSize, int onDiskDataSizeWithHeader, HFileContext fileContext, @@ -524,7 +523,6 @@ public class HFileBlock implements Cacheable { return this.allocator; } - @VisibleForTesting private void sanityCheckAssertion(long valueFromBuf, long valueFromField, String fieldName) throws IOException { if (valueFromBuf != valueFromField) { @@ -533,7 +531,6 @@ public class HFileBlock implements Cacheable { } } - @VisibleForTesting private void sanityCheckAssertion(BlockType valueFromBuf, BlockType valueFromField) throws IOException { if (valueFromBuf != valueFromField) { @@ -550,7 +547,6 @@ public class HFileBlock implements Cacheable { * thread-safe, because it alters the internal buffer pointer. * Used by tests only. */ - @VisibleForTesting void sanityCheck() throws IOException { // Duplicate so no side-effects ByteBuff dup = this.buf.duplicate().rewind(); @@ -839,7 +835,6 @@ public class HFileBlock implements Cacheable { /** * @param dataBlockEncoder data block encoding algorithm to use */ - @VisibleForTesting public Writer(HFileDataBlockEncoder dataBlockEncoder, HFileContext fileContext) { this(dataBlockEncoder, fileContext, ByteBuffAllocator.HEAP); } @@ -1402,7 +1397,6 @@ public class HFileBlock implements Cacheable { private long fileSize; /** The size of the header */ - @VisibleForTesting protected final int hdrSize; /** The filesystem used to access data */ @@ -1693,7 +1687,6 @@ public class HFileBlock implements Cacheable { * @param intoHeap allocate the ByteBuff of block from heap or off-heap. * @return the HFileBlock or null if there is a HBase checksum mismatch */ - @VisibleForTesting protected HFileBlock readBlockDataInternal(FSDataInputStream is, long offset, long onDiskSizeWithHeaderL, boolean pread, boolean verifyChecksum, boolean updateMetrics, boolean intoHeap) throws IOException { @@ -1851,7 +1844,6 @@ public class HFileBlock implements Cacheable { } /** An additional sanity-check in case no compression or encryption is being used. */ - @VisibleForTesting void sanityCheckUncompressed() throws IOException { if (onDiskSizeWithoutHeader != uncompressedSizeWithoutHeader + totalChecksumBytes()) { @@ -1973,7 +1965,6 @@ public class HFileBlock implements Cacheable { return DataBlockEncoding.NONE; } - @VisibleForTesting byte getChecksumType() { return this.fileContext.getChecksumType().getCode(); } @@ -1983,7 +1974,6 @@ public class HFileBlock implements Cacheable { } /** @return the size of data on disk + header. Excludes checksum. */ - @VisibleForTesting int getOnDiskDataSizeWithHeader() { return this.onDiskDataSizeWithHeader; } @@ -2022,7 +2012,6 @@ public class HFileBlock implements Cacheable { /** * Return the appropriate DUMMY_HEADER for the minor version */ - @VisibleForTesting // TODO: Why is this in here? byte[] getDummyHeaderForVersion() { return getDummyHeaderForVersion(this.fileContext.isUseHBaseChecksum()); @@ -2048,7 +2037,6 @@ public class HFileBlock implements Cacheable { * This is mostly helpful for debugging. This assumes that the block * has minor version > 0. */ - @VisibleForTesting static String toStringHeader(ByteBuff buf) throws IOException { byte[] magicBuf = new byte[Math.min(buf.limit() - buf.position(), BlockType.MAGIC_LENGTH)]; buf.get(magicBuf); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java index 2060b20de41..1ed1bb5d19c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java @@ -52,7 +52,6 @@ import org.apache.htrace.core.TraceScope; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; /** * Implementation that can handle all hfile versions of {@link HFile.Reader}. @@ -233,7 +232,6 @@ public abstract class HFileReaderImpl implements HFile.Reader, Configurable { return this.hfileContext.getCellComparator(); } - @VisibleForTesting public Compression.Algorithm getCompressionAlgorithm() { return trailer.getCompressionCodec(); } @@ -1627,7 +1625,6 @@ public abstract class HFileReaderImpl implements HFile.Reader, Configurable { * not completed, true otherwise */ @Override - @VisibleForTesting public boolean prefetchComplete() { return PrefetchExecutor.isCompleted(path); } @@ -1645,7 +1642,6 @@ public abstract class HFileReaderImpl implements HFile.Reader, Configurable { * @return Scanner on this file. */ @Override - @VisibleForTesting public HFileScanner getScanner(boolean cacheBlocks, final boolean pread) { return getScanner(cacheBlocks, pread, false); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java index 60aa65d4d74..b1a98487bdc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java @@ -54,8 +54,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * Common functionality needed by all versions of {@link HFile} writers. */ @@ -784,7 +782,6 @@ public class HFileWriterImpl implements HFile.Writer { } } - @VisibleForTesting public Cell getLastCell() { return lastCell; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java index b2016abe684..bc0df4306b6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java @@ -43,7 +43,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects; import org.apache.hbase.thirdparty.com.google.common.base.Objects; import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -636,12 +635,10 @@ public class LruBlockCache implements FirstLevelBlockCache { } } - @VisibleForTesting boolean isEvictionInProgress() { return evictionInProgress; } - @VisibleForTesting long getOverhead() { return overhead; } @@ -1173,7 +1170,6 @@ public class LruBlockCache implements FirstLevelBlockCache { } /** Clears the cache. Used in tests. */ - @VisibleForTesting public void clearCache() { this.map.clear(); this.elements.set(0); @@ -1184,7 +1180,6 @@ public class LruBlockCache implements FirstLevelBlockCache { * * @return the set of cached file names */ - @VisibleForTesting SortedSet getCachedFileNamesForTest() { SortedSet fileNames = new TreeSet<>(); for (BlockCacheKey cacheKey : map.keySet()) { @@ -1193,7 +1188,6 @@ public class LruBlockCache implements FirstLevelBlockCache { return fileNames; } - @VisibleForTesting public Map getEncodingCountsForTest() { Map counts = new EnumMap<>(DataBlockEncoding.class); for (LruCachedBlock block : map.values()) { @@ -1204,7 +1198,6 @@ public class LruBlockCache implements FirstLevelBlockCache { return counts; } - @VisibleForTesting Map getMapForTests() { return map; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ReaderContextBuilder.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ReaderContextBuilder.java index c58d5b8ce07..1f903cfbea6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ReaderContextBuilder.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ReaderContextBuilder.java @@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; import org.apache.hadoop.hbase.io.hfile.ReaderContext.ReaderType; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; /** * A builder that helps in building up the ReaderContext @@ -82,7 +81,6 @@ public class ReaderContextBuilder { return this; } - @VisibleForTesting public ReaderContextBuilder withFileSystemAndPath(FileSystem fs, Path filePath) throws IOException { this.withFileSystem(fs) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/TinyLfuBlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/TinyLfuBlockCache.java index a90c5a33db6..a0dc30c5242 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/TinyLfuBlockCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/TinyLfuBlockCache.java @@ -36,7 +36,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache; import org.apache.hadoop.util.StringUtils; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects; import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.yetus.audience.InterfaceAudience; @@ -69,7 +68,6 @@ public final class TinyLfuBlockCache implements FirstLevelBlockCache { private BlockCache victimCache; - @VisibleForTesting final Cache cache; /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java index a205d27f872..a84d8128207 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java @@ -49,7 +49,6 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Consumer; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.TableName; @@ -79,7 +78,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -119,7 +117,6 @@ public class BucketCache implements BlockCache, HeapSize { 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 @@ public class BucketCache implements BlockCache, HeapSize { transient final IOEngine ioEngine; // Store the block in this map before writing it to cache - @VisibleForTesting transient final RAMCache ramCache; // In this map, store the block's meta data like offset, length - @VisibleForTesting transient ConcurrentHashMap backingMap; /** @@ -161,9 +156,7 @@ public class BucketCache implements BlockCache, HeapSize { * WriterThread when it runs takes whatever has been recently added and 'drains' the entries * to the BucketCache. It then updates the ramCache and backingMap accordingly. */ - @VisibleForTesting transient final ArrayList> writerQueues = new ArrayList<>(); - @VisibleForTesting transient final WriterThread[] writerThreads; /** Volatile boolean to track if free space is in process or not */ @@ -185,7 +178,6 @@ public class BucketCache implements BlockCache, HeapSize { * bucket cache will skip some blocks when caching. If the flag is true, we * will wait until blocks are flushed to IOEngine. */ - @VisibleForTesting boolean wait_when_cache = false; private final BucketCacheStats cacheStats = new BucketCacheStats(); @@ -209,7 +201,6 @@ public class BucketCache implements BlockCache, HeapSize { * The purpose of this is to avoid freeing the block which is being read. *

*/ - @VisibleForTesting transient final IdReadWriteLock offsetLock; private final NavigableSet blocksByHFile = new ConcurrentSkipListSet<>((a, b) -> { @@ -352,14 +343,12 @@ public class BucketCache implements BlockCache, HeapSize { * Called by the constructor to start the writer threads. Used by tests that need to override * starting the threads. */ - @VisibleForTesting protected void startWriterThreads() { for (WriterThread thread : writerThreads) { thread.start(); } } - @VisibleForTesting boolean isCacheEnabled() { return this.cacheEnabled; } @@ -556,7 +545,6 @@ public class BucketCache implements BlockCache, HeapSize { return null; } - @VisibleForTesting void blockEvicted(BlockCacheKey cacheKey, BucketEntry bucketEntry, boolean decrementBlockNumber) { bucketAllocator.freeBlock(bucketEntry.offset()); realCacheSize.add(-1 * bucketEntry.getLength()); @@ -681,7 +669,6 @@ public class BucketCache implements BlockCache, HeapSize { return (long) Math.floor(bucketAllocator.getTotalSize() * acceptableFactor); } - @VisibleForTesting long getPartitionSize(float partitionFactor) { return (long) Math.floor(bucketAllocator.getTotalSize() * partitionFactor * minFactor); } @@ -876,7 +863,6 @@ public class BucketCache implements BlockCache, HeapSize { } // This handles flushing the RAM cache to IOEngine. - @VisibleForTesting class WriterThread extends Thread { private final BlockingQueue inputQueue; private volatile boolean writerEnabled = true; @@ -887,7 +873,6 @@ public class BucketCache implements BlockCache, HeapSize { } // Used for test - @VisibleForTesting void disableWriter() { this.writerEnabled = false; } @@ -947,7 +932,6 @@ public class BucketCache implements BlockCache, HeapSize { * interference expected. * @throws InterruptedException */ - @VisibleForTesting void doDrain(final List entries) throws InterruptedException { if (entries.isEmpty()) { return; @@ -1055,7 +1039,6 @@ public class BucketCache implements BlockCache, HeapSize { * @param q The queue to take from. * @return {@code receptacle} laden with elements taken from the queue or empty if none found. */ - @VisibleForTesting static List getRAMQueueEntries(BlockingQueue q, List receptacle) throws InterruptedException { // Clear sets all entries to null and sets size to 0. We retain allocations. Presume it @@ -1349,7 +1332,6 @@ public class BucketCache implements BlockCache, HeapSize { /** * Block Entry stored in the memory with key,data and so on */ - @VisibleForTesting static class RAMQueueEntry { private final BlockCacheKey key; private final Cacheable data; @@ -1531,7 +1513,6 @@ public class BucketCache implements BlockCache, HeapSize { return null; } - @VisibleForTesting public int getRpcRefCount(BlockCacheKey cacheKey) { BucketEntry bucketEntry = backingMap.get(cacheKey); if (bucketEntry != null) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java index 2cdfc80a39c..e4a2c0b1aea 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java @@ -27,7 +27,6 @@ import java.nio.channels.ClosedChannelException; import java.nio.channels.FileChannel; import java.util.Arrays; import java.util.concurrent.locks.ReentrantLock; - import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException; import org.apache.hadoop.hbase.io.hfile.Cacheable; import org.apache.hadoop.hbase.nio.ByteBuff; @@ -36,7 +35,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; /** @@ -154,7 +152,6 @@ public class FileIOEngine extends PersistentIOEngine { return be.wrapAsCacheable(dstBuff); } - @VisibleForTesting void closeFileChannels() { for (FileChannel fileChannel: fileChannels) { try { @@ -283,12 +280,10 @@ public class FileIOEngine extends PersistentIOEngine { return fileNum; } - @VisibleForTesting FileChannel[] getFileChannels() { return fileChannels; } - @VisibleForTesting void refreshFileConnection(int accessFileNum, IOException ioe) throws IOException { ReentrantLock channelLock = channelLocks[accessFileNum]; channelLock.lock(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java index eab2a0ec85c..f34cad5f60c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java @@ -35,7 +35,7 @@ import org.apache.hadoop.security.authorize.ServiceAuthorizationManager; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService; import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor; import org.apache.hbase.thirdparty.com.google.protobuf.Message; @@ -124,7 +124,7 @@ public class NettyRpcServer extends RpcServer { this.scheduler.init(new RpcSchedulerContext(this)); } - @VisibleForTesting + @InterfaceAudience.Private protected NettyRpcServerPreambleHandler createNettyRpcServerPreambleHandler() { return new NettyRpcServerPreambleHandler(NettyRpcServer.this); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerPreambleHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerPreambleHandler.java index bac19f1bb11..855cf2fda4d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerPreambleHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerPreambleHandler.java @@ -26,7 +26,6 @@ import org.apache.hbase.thirdparty.io.netty.channel.SimpleChannelInboundHandler; import java.nio.ByteBuffer; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; /** * Handle connection preamble. @@ -58,7 +57,6 @@ class NettyRpcServerPreambleHandler extends SimpleChannelInboundHandler p.remove("preambleDecoder"); } - @VisibleForTesting protected NettyServerRpcConnection createNettyServerRpcConnection(Channel channel) { return new NettyServerRpcConnection(rpcServer, channel); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java index 7bae06f601b..ca8593ee3d5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java @@ -33,7 +33,6 @@ import java.util.Locale; import java.util.Map; import java.util.Optional; import java.util.concurrent.atomic.LongAdder; - import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CallQueueTooBigException; @@ -68,7 +67,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.gson.Gson; import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService; import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor; @@ -551,7 +549,6 @@ public abstract class RpcServer implements RpcServerInterface, * @param strParam stringifiedParam to be truncated * @return truncated trace log string */ - @VisibleForTesting String truncateTraceLog(String strParam) { if (LOG.isTraceEnabled()) { int traceLogMaxLength = getConf().getInt(TRACE_LOG_MAX_LENGTH, DEFAULT_TRACE_LOG_MAX_LENGTH); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServerInterface.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServerInterface.java index 99e01885b5b..e06daacf5fe 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServerInterface.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServerInterface.java @@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.ipc; import java.io.IOException; import java.net.InetSocketAddress; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.io.ByteBuffAllocator; @@ -32,7 +31,6 @@ import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.security.authorize.PolicyProvider; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService; import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor; import org.apache.hbase.thirdparty.com.google.protobuf.Message; @@ -86,7 +84,6 @@ public interface RpcServerInterface { * Refresh authentication manager policy. * @param pp */ - @VisibleForTesting void refreshAuthManager(Configuration conf, PolicyProvider pp); RpcScheduler getScheduler(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CachedClusterId.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CachedClusterId.java index 2e7baae27c3..2f75560dae8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CachedClusterId.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CachedClusterId.java @@ -32,7 +32,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; /** @@ -157,7 +156,6 @@ public class CachedClusterId { return getClusterId(); } - @VisibleForTesting public int getCacheStats() { return cacheMisses.get(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index 7adbeb0ce87..7d29ed66882 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.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hbase.thirdparty.com.google.common.collect.Maps; import org.apache.hbase.thirdparty.com.google.common.collect.Sets; @@ -267,7 +266,7 @@ public class HMaster extends HRegionServer implements MasterServices { // Tracker for meta location, if any client ZK quorum specified private MetaLocationSyncer metaLocationSyncer; // Tracker for active master location, if any client ZK quorum specified - @VisibleForTesting + @InterfaceAudience.Private MasterAddressSyncer masterAddressSyncer; // Tracker for auto snapshot cleanup state SnapshotCleanupTracker snapshotCleanupTracker; @@ -621,7 +620,7 @@ public class HMaster extends HRegionServer implements MasterServices { } } - @VisibleForTesting + @InterfaceAudience.Private public MasterRpcServices getMasterRpcServices() { return (MasterRpcServices)rpcServices; } @@ -751,7 +750,7 @@ public class HMaster extends HRegionServer implements MasterServices { } // Will be overriden in test to inject customized AssignmentManager - @VisibleForTesting + @InterfaceAudience.Private protected AssignmentManager createAssignmentManager(MasterServices master) { return new AssignmentManager(master); } @@ -1117,7 +1116,7 @@ public class HMaster extends HRegionServer implements MasterServices { * @return True if meta is UP and online and startup can progress. Otherwise, meta is not online * and we will hold here until operator intervention. */ - @VisibleForTesting + @InterfaceAudience.Private public boolean waitForMetaOnline() { return isRegionOnline(RegionInfoBuilder.FIRST_META_REGIONINFO); } @@ -1188,7 +1187,7 @@ public class HMaster extends HRegionServer implements MasterServices { * Adds the {@code MasterQuotasObserver} to the list of configured Master observers to * automatically remove quotas for a table when that table is deleted. */ - @VisibleForTesting + @InterfaceAudience.Private public void updateConfigurationForQuotasObserver(Configuration conf) { // We're configured to not delete quotas on table deletion, so we don't need to add the obs. if (!conf.getBoolean( @@ -1221,7 +1220,7 @@ public class HMaster extends HRegionServer implements MasterServices { * Will be overridden in tests. *

*/ - @VisibleForTesting + @InterfaceAudience.Private protected ServerManager createServerManager(final MasterServices master) throws IOException { // We put this out here in a method so can do a Mockito.spy and stub it out // w/ a mocked up ServerManager. @@ -1235,7 +1234,7 @@ public class HMaster extends HRegionServer implements MasterServices { } // Will be overridden in tests - @VisibleForTesting + @InterfaceAudience.Private protected void initClusterSchemaService() throws IOException, InterruptedException { this.clusterSchemaService = new ClusterSchemaServiceImpl(this); this.clusterSchemaService.startAsync(); @@ -1912,7 +1911,7 @@ public class HMaster extends HRegionServer implements MasterServices { // Public so can be accessed by tests. Blocks until move is done. // Replace with an async implementation from which you can get // a success/failure result. - @VisibleForTesting + @InterfaceAudience.Private public void move(final byte[] encodedRegionName, byte[] destServerName) throws IOException { RegionState regionState = assignmentManager.getRegionStates(). getRegionState(Bytes.toString(encodedRegionName)); @@ -2740,7 +2739,7 @@ public class HMaster extends HRegionServer implements MasterServices { } } - @VisibleForTesting + @InterfaceAudience.Private protected void checkServiceStarted() throws ServerNotRunningYetException { if (!serviceStarted) { throw new ServerNotRunningYetException("Server is not running yet"); @@ -2795,7 +2794,7 @@ public class HMaster extends HRegionServer implements MasterServices { return maintenanceMode; } - @VisibleForTesting + @InterfaceAudience.Private public void setInitialized(boolean isInitialized) { procedureExecutor.getEnvironment().setEventReady(initialized, isInitialized); } @@ -3643,7 +3642,7 @@ public class HMaster extends HRegionServer implements MasterServices { /** * This method modifies the master's configuration in order to inject replication-related features */ - @VisibleForTesting + @InterfaceAudience.Private public static void decorateMasterConfiguration(Configuration conf) { String plugins = conf.get(HBASE_MASTER_LOGCLEANER_PLUGINS); String cleanerClass = ReplicationLogCleaner.class.getCanonicalName(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java index 0bbfa4a180e..ca7eb909859 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.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * This class abstracts a bunch of operations the HMaster needs to interact with * the underlying file system like creating the initial layout, checking file @@ -293,7 +291,6 @@ public class MasterFileSystem { * Make sure the hbase temp directory exists and is empty. * NOTE that this method is only executed once just after the master becomes the active one. */ - @VisibleForTesting void checkTempDir(final Path tmpdir, final Configuration c, final FileSystem fs) throws IOException { // If the temp directory exists, clear the content (left over, from the previous run) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java index c5f0f3c4bca..f24ecd46d48 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.AccessChecker; 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 @@ public interface MasterServices extends Server { /** * @return Tripped when Master has finished initialization. */ - @VisibleForTesting public ProcedureEvent getInitializedEvent(); /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java index 6001c8f9a98..61b432710d8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java @@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.wal.WALSplitter; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; /** * This class abstracts a bunch of operations the HMaster needs @@ -66,7 +65,6 @@ public class MasterWalManager { /** * Filter *out* WAL files that are for the hbase:meta Region; i.e. return user-space WALs only. */ - @VisibleForTesting public final static PathFilter NON_META_FILTER = new PathFilter() { @Override public boolean accept(Path p) { @@ -124,7 +122,6 @@ public class MasterWalManager { } } - @VisibleForTesting SplitLogManager getSplitLogManager() { return this.splitLogManager; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java index d1d935247a7..9e666c56a89 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.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; @@ -276,7 +275,6 @@ public class ServerManager { } } - @VisibleForTesting public void regionServerReport(ServerName sn, ServerMetrics sl) throws YouAreDeadException { checkIsDead(sn, "REPORT"); @@ -428,13 +426,11 @@ public class ServerManager { * Adds the onlineServers list. onlineServers should be locked. * @param serverName The remote servers name. */ - @VisibleForTesting void recordNewServerWithLock(final ServerName serverName, final ServerMetrics sl) { LOG.info("Registering regionserver=" + serverName); this.onlineServers.put(serverName, sl); } - @VisibleForTesting public ConcurrentNavigableMap getFlushedSequenceIdByRegion() { return flushedSequenceIdByRegion; } @@ -570,7 +566,7 @@ public class ServerManager { * going down or we already have queued an SCP for this server or SCP processing is * currently disabled because we are in startup phase). */ - @VisibleForTesting // Redo test so we can make this protected. + // Redo test so we can make this protected. public synchronized long expireServer(final ServerName serverName) { return expireServer(serverName, false); @@ -629,7 +625,6 @@ public class ServerManager { * Called when server has expired. */ // Locking in this class needs cleanup. - @VisibleForTesting public synchronized void moveFromOnlineToDeadServers(final ServerName sn) { synchronized (this.onlineServers) { boolean online = this.onlineServers.containsKey(sn); @@ -996,7 +991,6 @@ public class ServerManager { flushedSequenceIdByRegion.remove(encodedName); } - @VisibleForTesting public boolean isRegionInServerManagerStates(final RegionInfo hri) { final byte[] encodedName = hri.getEncodedNameAsBytes(); return (storeFlushedSequenceIdsByRegion.containsKey(encodedName) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java index 3e0c7460eaf..465a59367ca 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java @@ -55,7 +55,6 @@ import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; /** * Distributes the task of log splitting to the available region servers. @@ -104,7 +103,6 @@ public class SplitLogManager { private long unassignedTimeout; private long lastTaskCreateTime = Long.MAX_VALUE; - @VisibleForTesting final ConcurrentMap tasks = new ConcurrentHashMap<>(); private TimeoutMonitor timeoutMonitor; @@ -165,7 +163,6 @@ public class SplitLogManager { * {@link org.apache.hadoop.hbase.wal.WALSplitter#split(Path, Path, Path, FileSystem, * Configuration, org.apache.hadoop.hbase.wal.WALFactory)} for tests. */ - @VisibleForTesting public static FileStatus[] getFileList(final Configuration conf, final List logDirs, final PathFilter filter) throws IOException { @@ -375,7 +372,6 @@ public class SplitLogManager { } } - @VisibleForTesting ConcurrentMap getTasks() { return tasks; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitWALManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitWALManager.java index 9ff84dc942e..48c19c24ed0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitWALManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitWALManager.java @@ -43,11 +43,10 @@ import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException; import org.apache.hadoop.hbase.util.CommonFSUtils; import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; import org.apache.hadoop.hbase.wal.WALSplitUtil; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; -import org.apache.hbase.thirdparty.com.google.common.collect.Lists; /** * Create {@link SplitWALProcedure} for each WAL which need to split. Manage the workers for each @@ -145,7 +144,6 @@ public class SplitWALManager { return !fs.exists(new Path(rootDir, walPath)); } - @VisibleForTesting List createSplitWALProcedures(List splittingWALs, ServerName crashedServer) { return splittingWALs.stream() diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java index 25c94ba5bbc..fe304cebf06 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java @@ -27,14 +27,11 @@ import org.apache.hadoop.hbase.procedure2.ProcedureMetrics; import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException; import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation; -import org.apache.yetus.audience.InterfaceAudience; - -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AssignRegionStateData; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionTransitionState; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode; +import org.apache.yetus.audience.InterfaceAudience; /** * Leave here only for checking if we can successfully start the master. @@ -143,7 +140,6 @@ public class AssignProcedure extends RegionTransitionProcedure { return env.getAssignmentManager().getAssignmentManagerMetrics().getAssignProcMetrics(); } - @VisibleForTesting @Override public void setProcId(long procId) { super.setProcId(procId); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java index 355dfde013a..4d0e165456d 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.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode; @@ -181,7 +179,6 @@ public class AssignmentManager { this(master, new RegionStateStore(master)); } - @VisibleForTesting AssignmentManager(final MasterServices master, final RegionStateStore stateStore) { this.master = master; this.regionStateStore = stateStore; @@ -513,7 +510,6 @@ public class AssignmentManager { * This is a bit dirty, should be reconsidered after we decide whether to keep the * {@link #processOfflineRegions()} method. */ - @VisibleForTesting public void wakeMetaLoadedEvent() { metaLoadEvent.wake(getProcedureScheduler()); assert isMetaLoaded() : "expected meta to be loaded"; @@ -783,7 +779,6 @@ public class AssignmentManager { return createRoundRobinAssignProcedures(hris, null); } - @VisibleForTesting static int compare(TransitRegionStateProcedure left, TransitRegionStateProcedure right) { if (left.getRegion().isMetaRegion()) { if (right.getRegion().isMetaRegion()) { @@ -1368,7 +1363,6 @@ public class AssignmentManager { private int totalRITsTwiceThreshold = 0; private int totalRITs = 0; - @VisibleForTesting public RegionInTransitionStat(final Configuration conf) { this.ritThreshold = conf.getInt(METRICS_RIT_STUCK_WARNING_THRESHOLD, DEFAULT_RIT_STUCK_WARNING_THRESHOLD); @@ -2270,7 +2264,6 @@ public class AssignmentManager { .collect(Collectors.toList()); } - @VisibleForTesting MasterServices getMaster() { return master; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java index 5e06a44912b..80a61dae903 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.hadoop.hbase.wal.WALSplitUtil; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos; @@ -752,7 +752,6 @@ public class MergeTableRegionsProcedure /** * @return The merged region. Maybe be null if called to early or we failed. */ - @VisibleForTesting RegionInfo getMergedRegion() { return this.mergedRegion; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateNode.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateNode.java index b7fcdab96b9..d04dbef66ba 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateNode.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateNode.java @@ -36,8 +36,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * Current Region State. Most fields are synchronized with meta region, i.e, we will update meta * immediately after we modify this RegionStateNode, and usually under the lock. The only exception @@ -77,7 +75,6 @@ public class RegionStateNode implements Comparable { } } - @VisibleForTesting final Lock lock = new ReentrantLock(); private final RegionInfo regionInfo; private final ProcedureEvent event; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java index c8b0e351f4e..4da9493386a 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.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; @@ -464,7 +463,6 @@ public class RegionStateStore { qualifiers.stream().map(Bytes::toStringBinary).collect(Collectors.joining(", "))); } - @VisibleForTesting static Put addMergeRegions(Put put, Collection 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 06378002ecb..147a112152b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java @@ -45,8 +45,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * RegionStates contains a set of Maps that describes the in-memory state of the AM, with * the regions available in the system, the region in transition, the offline regions and @@ -115,7 +113,6 @@ public class RegionStates { serverMap.clear(); } - @VisibleForTesting public boolean isRegionInRegionStates(final RegionInfo hri) { return (regionsMap.containsKey(hri.getRegionName()) || regionInTransition.containsKey(hri) || regionOffline.containsKey(hri)); @@ -124,7 +121,6 @@ public class RegionStates { // ========================================================================== // RegionStateNode helpers // ========================================================================== - @VisibleForTesting RegionStateNode createRegionStateNode(RegionInfo regionInfo) { synchronized (regionsMapLock) { RegionStateNode node = regionsMap.computeIfAbsent(regionInfo.getRegionName(), @@ -764,7 +760,6 @@ public class RegionStates { /** * @return Pertinent ServerStateNode or NULL if none found (Do not make modifications). */ - @VisibleForTesting public ServerStateNode getServerNode(final ServerName serverName) { return serverMap.get(serverName); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java index 2e767718367..bc46e19978d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java @@ -31,10 +31,9 @@ import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException; import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation; import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure; import org.apache.hadoop.hbase.procedure2.RemoteProcedureException; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionTransitionState; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode; +import org.apache.yetus.audience.InterfaceAudience; /** * Leave here only for checking if we can successfully start the master. @@ -62,12 +61,10 @@ public abstract class RegionTransitionProcedure extends ProcedureThis is a best effort load balancer. Given a Cost function F(C) => x It will * randomly try and mutate the cluster to Cprime. If F(Cprime) < F(C) then the @@ -365,7 +362,6 @@ public class StochasticLoadBalancer extends BaseLoadBalancer { 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 4331d490c28..9416e5a12a6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java @@ -28,7 +28,6 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -42,7 +41,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet; import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; @@ -373,7 +371,6 @@ public abstract class CleanerChore extends Schedu } } - @VisibleForTesting int getChorePoolSize() { return pool.getSize(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java index 6926f12c49e..ff288572e25 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java @@ -39,7 +39,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; /** * This Chore, every time it runs, will clear the HFiles in the hfile archive * folder that are deletable for each HFile cleaner in the chain. @@ -82,12 +81,10 @@ public class HFileCleaner extends CleanerChore public static final String HFILE_DELETE_THREAD_TIMEOUT_MSEC = "hbase.regionserver.hfilecleaner.thread.timeout.msec"; - @VisibleForTesting static final long DEFAULT_HFILE_DELETE_THREAD_TIMEOUT_MSEC = 60 * 1000L; public static final String HFILE_DELETE_THREAD_CHECK_INTERVAL_MSEC = "hbase.regionserver.hfilecleaner.thread.check.interval.msec"; - @VisibleForTesting static final long DEFAULT_HFILE_DELETE_THREAD_CHECK_INTERVAL_MSEC = 1000L; private static final Logger LOG = LoggerFactory.getLogger(HFileCleaner.class); @@ -383,42 +380,34 @@ public class HFileCleaner extends CleanerChore } } - @VisibleForTesting public List getCleanerThreads() { return threads; } - @VisibleForTesting public long getNumOfDeletedLargeFiles() { return deletedLargeFiles.get(); } - @VisibleForTesting public long getNumOfDeletedSmallFiles() { return deletedSmallFiles.get(); } - @VisibleForTesting public long getLargeQueueInitSize() { return largeQueueInitSize; } - @VisibleForTesting public long getSmallQueueInitSize() { return smallQueueInitSize; } - @VisibleForTesting public long getThrottlePoint() { return throttlePoint; } - @VisibleForTesting long getCleanerThreadTimeoutMsec() { return cleanerThreadTimeoutMsec; } - @VisibleForTesting long getCleanerThreadCheckIntervalMsec() { return cleanerThreadCheckIntervalMsec; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java index 5fa115c9b8b..f65713ebf26 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java @@ -27,7 +27,6 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -41,7 +40,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; /** @@ -59,7 +57,6 @@ public class LogCleaner extends CleanerChore public static final String OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC = "hbase.oldwals.cleaner.thread.timeout.msec"; - @VisibleForTesting static final long DEFAULT_OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC = 60 * 1000L; private final LinkedBlockingQueue pendingDelete; @@ -138,12 +135,10 @@ public class LogCleaner extends CleanerChore interruptOldWALsCleaner(); } - @VisibleForTesting int getSizeOfCleaners() { return oldWALsCleaner.size(); } - @VisibleForTesting long getCleanerThreadTimeoutMsec() { return cleanerThreadTimeoutMsec; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/CatalogJanitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/CatalogJanitor.java index 6123f6b6c3b..45b6a746a54 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.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * A janitor for the catalog tables. Scans the hbase:meta catalog table on a period. * Makes a lastReport on state of hbase:meta. Looks for unused regions to garbage collect. Scan of @@ -218,7 +216,6 @@ public class CatalogJanitor extends ScheduledChore { * @return Return generated {@link Report} */ // will be override in tests. - @VisibleForTesting protected Report scanForReport() throws IOException { ReportMakingVisitor visitor = new ReportMakingVisitor(this.services); // Null tablename means scan all of meta. @@ -304,7 +301,6 @@ public class CatalogJanitor extends ScheduledChore { } } - @VisibleForTesting static boolean cleanParent(MasterServices services, RegionInfo parent, Result rowContent) throws IOException { // Check whether it is a merged region and if it is clean of references. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/MetaFixer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/MetaFixer.java index f9dc1ccb5aa..4a5aa0a1b8e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/MetaFixer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/MetaFixer.java @@ -48,11 +48,9 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap; import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap; - /** * Server-side fixing of bad or inconsistent state in hbase:meta. * Distinct from MetaTableAccessor because {@link MetaTableAccessor} is about low-level @@ -261,7 +259,6 @@ public class MetaFixer { * @param maxMergeCount Maximum regions to merge at a time (avoid merging * 100k regions in one go!) */ - @VisibleForTesting static List> calculateMerges(int maxMergeCount, List> overlaps) { if (overlaps.isEmpty()) { @@ -333,7 +330,6 @@ public class MetaFixer { * @return Either a or b, whichever has the * endkey that is furthest along in the Table. */ - @VisibleForTesting static RegionInfo getRegionInfoWithLargestEndKey(RegionInfo a, RegionInfo b) { if (a == null) { // b may be null. @@ -361,7 +357,6 @@ public class MetaFixer { * @return True if an overlap found between passed in ri and * the pair. Does NOT check the pairs themselves overlap. */ - @VisibleForTesting static boolean isOverlap(RegionInfo ri, Pair pair) { if (ri == null || pair == null) { // Can't be an overlap in either of these cases. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockManager.java index 26b838dcefb..aaf51526f4f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockManager.java @@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.util.NonceKey; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; /** * Functions to acquire lock on table/namespace/regions. @@ -192,7 +191,6 @@ public final class LockManager { return "MasterLock: proc = " + proc.toString(); } - @VisibleForTesting LockProcedure getProc() { return proc; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java index e027a738fc7..bd6d44ca166 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.rsgroup.RSGroupInfo; 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 @@ public abstract class AbstractStateMachineNamespaceProcedure 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 9ffe3c6e22f..2313e70f75b 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.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; @@ -421,7 +420,6 @@ public class CreateTableProcedure return !getTableName().isSystemTable(); } - @VisibleForTesting RegionInfo getFirstRegionInfo() { if (newRegions == null || newRegions.isEmpty()) { return null; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java index c470c428125..456660d3757 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.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * ProcedureScheduler for the Master Procedures. * This ProcedureScheduler tries to provide to the ProcedureExecutor procedures @@ -659,7 +657,6 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler { * @return true if deletion succeeded, false otherwise meaning that there are * other new operations pending for that table (e.g. a new create). */ - @VisibleForTesting boolean markTableAsDeleted(final TableName table, final Procedure procedure) { schedLock(); try { @@ -1015,7 +1012,6 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler { /** * For debugging. Expensive. */ - @VisibleForTesting public String dumpLocks() throws IOException { schedLock(); try { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java index 1942ed6e8ab..2f990cb0b06 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.hadoop.ipc.RemoteException; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap; import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; + import org.apache.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 @@ public class RSProcedureDispatcher } // 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 4ae408f417d..f8822311efa 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitWALProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitWALProcedure.java @@ -16,6 +16,7 @@ * limitations under the License. */ package org.apache.hadoop.hbase.master.procedure; + import java.io.IOException; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.ServerName; @@ -30,7 +31,7 @@ import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos; @@ -163,8 +164,7 @@ public class SplitWALProcedure 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 290e0c19df1..26eceb90745 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.List; - import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotDisabledException; @@ -37,7 +36,7 @@ import org.apache.hadoop.hbase.util.ModifyRegionUtils; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos; @@ -318,7 +317,6 @@ public class TruncateTableProcedure } } - @VisibleForTesting RegionInfo getFirstRegionInfo() { if (regions == null || regions.isEmpty()) { return null; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/region/MasterRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/region/MasterRegion.java index 688a5497ddc..c2188b4dc0d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/region/MasterRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/region/MasterRegion.java @@ -50,7 +50,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.math.IntMath; /** @@ -96,10 +95,8 @@ public final class MasterRegion { private final WALFactory walFactory; - @VisibleForTesting final HRegion region; - @VisibleForTesting final MasterRegionFlusherAndCompactor flusherAndCompactor; private MasterRegionWALRoller walRoller; @@ -141,17 +138,14 @@ public final class MasterRegion { return region.getScanner(scan); } - @VisibleForTesting public FlushResult flush(boolean force) throws IOException { return region.flush(force); } - @VisibleForTesting public void requestRollAll() { walRoller.requestRollAll(); } - @VisibleForTesting public void waitUntilWalRollFinished() throws InterruptedException { walRoller.waitUntilWalRollFinished(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java index fc254a397a0..8985d2f629e 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.apache.yetus.audience.InterfaceAudience; 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 @@ public abstract class AbstractPeerProcedure extends AbstractPeerNoLockPr } // 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 8dd329f8569..7401c4b8d54 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.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerModificationState; /** @@ -133,7 +131,6 @@ public abstract class ModifyPeerProcedure extends AbstractPeerProcedure @@ -289,7 +287,6 @@ public class SyncReplicationReplayWALManager { 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 358fd5e3492..289e012c28a 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.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerSyncReplicationStateTransitionState; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.TransitPeerSyncReplicationStateStateData; @@ -110,7 +108,6 @@ public class TransitPeerSyncReplicationStateProcedure 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 @@ public class TransitPeerSyncReplicationStateProcedure } } - @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 @@ public class TransitPeerSyncReplicationStateProcedure addChildProcedure(new RecoverStandbyProcedure(peerId, serial)); } - @VisibleForTesting protected void setPeerNewSyncReplicationState(MasterProcedureEnv env) throws ReplicationException { if (toState.equals(SyncReplicationState.STANDBY) || @@ -223,12 +217,10 @@ public class TransitPeerSyncReplicationStateProcedure 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 039988af186..fe46e4c9711 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.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; /** @@ -271,7 +270,6 @@ public class SnapshotFileCache implements Stoppable { this.snapshots.putAll(newSnapshots); } - @VisibleForTesting List getSnapshotsInProgress() throws IOException { List snapshotInProgress = Lists.newArrayList(); // only add those files to the cache, but not to the known snapshots diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java index 8f57e6ebc28..30107790677 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java @@ -92,7 +92,6 @@ import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; @@ -196,7 +195,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable * @param coordinator procedure coordinator instance. exposed for testing. * @param pool HBase ExecutorServcie instance, exposed for testing. */ - @VisibleForTesting + @InterfaceAudience.Private SnapshotManager(final MasterServices master, ProcedureCoordinator coordinator, ExecutorService pool, int sentinelCleanInterval) throws IOException, UnsupportedOperationException { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCleanerChore.java index 82a08a9fb84..13f43fcf083 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.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.SetMultimap; /** @@ -101,7 +100,6 @@ public class MobFileCleanerChore extends ScheduledChore { } } - @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 dd5d2898ea0..7fe2d0d0ace 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.apache.yetus.audience.InterfaceAudience; 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 @@ public class MobFileCompactionChore extends ScheduledChore { } - @VisibleForTesting public MobFileCompactionChore(Configuration conf, int batchSize) { this.regionBatchSize = batchSize; } @@ -143,7 +139,6 @@ public class MobFileCompactionChore extends ScheduledChore { } } - @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 9e58daa3f74..720f25b57d9 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.RejectedExecutionException; 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 @@ public class ProcedureCoordinator { * * @param pool Used for executing procedures. */ - @VisibleForTesting // Only used in tests. SimpleMasterProcedureManager is a test class. + // Only used in tests. SimpleMasterProcedureManager is a test class. public ProcedureCoordinator(ProcedureCoordinatorRpcs rpcs, ThreadPoolExecutor pool) { this(rpcs, pool, TIMEOUT_MILLIS_DEFAULT, WAKE_MILLIS_DEFAULT); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/store/region/RegionProcedureStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/store/region/RegionProcedureStore.java index 36a919d8f19..301b6053fb2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/store/region/RegionProcedureStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/store/region/RegionProcedureStore.java @@ -63,7 +63,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet; import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos; @@ -87,7 +86,6 @@ public class RegionProcedureStore extends ProcedureStoreBase { private final LeaseRecovery leaseRecovery; - @VisibleForTesting final MasterRegion region; private int numThreads; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/AverageIntervalRateLimiter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/AverageIntervalRateLimiter.java index b2457959190..d10e6eacc7c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/AverageIntervalRateLimiter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/AverageIntervalRateLimiter.java @@ -13,7 +13,6 @@ package org.apache.hadoop.hbase.quotas; import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; /** * This limiter will refill resources at every TimeUnit/resources interval. For example: For a @@ -62,13 +61,11 @@ public class AverageIntervalRateLimiter extends RateLimiter { } // This method is for strictly testing purpose only - @VisibleForTesting @Override public void setNextRefillTime(long nextRefillTime) { this.nextRefillTime = nextRefillTime; } - @VisibleForTesting @Override public long getNextRefillTime() { return this.nextRefillTime; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FileArchiverNotifierFactoryImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FileArchiverNotifierFactoryImpl.java index bbe53b267af..b7cd26f1c25 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.TableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * A factory for getting instances of {@link FileArchiverNotifier}. */ @@ -47,12 +45,10 @@ public final class FileArchiverNotifierFactoryImpl implements FileArchiverNotifi return CURRENT_INSTANCE; } - @VisibleForTesting static void setInstance(FileArchiverNotifierFactory inst) { CURRENT_INSTANCE = Objects.requireNonNull(inst); } - @VisibleForTesting static void reset() { CURRENT_INSTANCE = DEFAULT_INSTANCE; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FileSystemUtilizationChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FileSystemUtilizationChore.java index edda4dfeafd..501ad8b8b2e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FileSystemUtilizationChore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FileSystemUtilizationChore.java @@ -161,7 +161,7 @@ public class FileSystemUtilizationChore extends ScheduledChore { return regionSize; } - // VisibleForTesting + // visible for testing RegionSizeStore getRegionSizeStore() { return rs.getRegionServerSpaceQuotaManager().getRegionSizeStore(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FixedIntervalRateLimiter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FixedIntervalRateLimiter.java index e67eda5c5bd..50f40afbe3c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FixedIntervalRateLimiter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FixedIntervalRateLimiter.java @@ -13,7 +13,6 @@ package org.apache.hadoop.hbase.quotas; import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; /** * With this limiter resources will be refilled only after a fixed interval of time. @@ -44,13 +43,11 @@ public class FixedIntervalRateLimiter extends RateLimiter { } // This method is for strictly testing purpose only - @VisibleForTesting @Override public void setNextRefillTime(long nextRefillTime) { this.nextRefillTime = nextRefillTime; } - @VisibleForTesting @Override public long getNextRefillTime() { return this.nextRefillTime; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java index d80134a364b..ff4c5191c94 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java @@ -48,7 +48,6 @@ import org.apache.yetus.audience.InterfaceStability; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.collect.HashMultimap; import org.apache.hbase.thirdparty.com.google.common.collect.Maps; import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat; @@ -676,7 +675,6 @@ public class MasterQuotaManager implements RegionStateListener { } } - @VisibleForTesting void initializeRegionSizes() { assert regionSizes == null; this.regionSizes = new ConcurrentHashMap<>(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaCache.java index 1c97b2012c0..bd1bff157cd 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 static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsent; import org.apache.hadoop.hbase.ClusterMetrics.Option; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import java.io.IOException; import java.util.ArrayList; @@ -180,32 +179,26 @@ public class QuotaCache implements Stoppable { return computeIfAbsent(quotasMap, key, QuotaState::new, this::triggerCacheRefresh); } - @VisibleForTesting void triggerCacheRefresh() { refreshChore.triggerNow(); } - @VisibleForTesting long getLastUpdate() { return refreshChore.lastUpdate; } - @VisibleForTesting Map getNamespaceQuotaCache() { return namespaceQuotaCache; } - @VisibleForTesting Map getRegionServerQuotaCache() { return regionServerQuotaCache; } - @VisibleForTesting Map getTableQuotaCache() { return tableQuotaCache; } - @VisibleForTesting Map getUserQuotaCache() { return userQuotaCache; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaObserverChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaObserverChore.java index 425c24013aa..0a8bfe18abb 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaObserverChore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaObserverChore.java @@ -25,7 +25,6 @@ import java.util.Objects; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ScheduledChore; import org.apache.hadoop.hbase.Stoppable; @@ -41,10 +40,11 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + import org.apache.hbase.thirdparty.com.google.common.collect.HashMultimap; import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; import org.apache.hbase.thirdparty.com.google.common.collect.Multimap; + import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota; /** @@ -521,12 +521,10 @@ public class QuotaObserverChore extends ScheduledChore { } } - @VisibleForTesting QuotaSnapshotStore getTableSnapshotStore() { return tableSnapshotStore; } - @VisibleForTesting QuotaSnapshotStore getNamespaceSnapshotStore() { return namespaceSnapshotStore; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RateLimiter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RateLimiter.java index 852d8a68f02..a29b90d5217 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RateLimiter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RateLimiter.java @@ -23,8 +23,6 @@ import java.util.concurrent.TimeUnit; import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * Simple rate limiter. * @@ -230,9 +228,8 @@ public abstract class RateLimiter { } // These two method are for strictly testing purpose only - @VisibleForTesting + public abstract void setNextRefillTime(long nextRefillTime); - @VisibleForTesting public abstract long getNextRefillTime(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerRpcQuotaManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerRpcQuotaManager.java index 0f96de53565..8eee70efabe 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.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; /** @@ -83,7 +83,6 @@ public class RegionServerRpcQuotaManager { } } - @VisibleForTesting protected boolean isRpcThrottleEnabled() { return rpcThrottleEnabled; } @@ -108,7 +107,6 @@ public class RegionServerRpcQuotaManager { } } - @VisibleForTesting QuotaCache getQuotaCache() { return quotaCache; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerSpaceQuotaManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerSpaceQuotaManager.java index b9797bc8b7d..81e7e87603c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerSpaceQuotaManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerSpaceQuotaManager.java @@ -24,16 +24,14 @@ import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicReference; import java.util.Map.Entry; - import org.apache.hadoop.hbase.TableName; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus; import org.apache.hadoop.hbase.regionserver.RegionServerServices; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; @@ -68,7 +66,6 @@ public class RegionServerSpaceQuotaManager { this(rsServices, SpaceViolationPolicyEnforcementFactory.getInstance()); } - @VisibleForTesting RegionServerSpaceQuotaManager( RegionServerServices rsServices, SpaceViolationPolicyEnforcementFactory factory) { this.rsServices = Objects.requireNonNull(rsServices); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java index 60cc92f428c..0d5f0b5965f 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.conf.Configuration; 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 @@ public abstract class AbstractMemStore implements MemStore { return comparator; } - @VisibleForTesting MutableSegment getActive() { return active; } - @VisibleForTesting ImmutableSegment getSnapshot() { return snapshot; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AnnotationReadingPriorityFunction.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AnnotationReadingPriorityFunction.java index 2dbc1066be7..df6295702a3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AnnotationReadingPriorityFunction.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AnnotationReadingPriorityFunction.java @@ -20,14 +20,18 @@ package org.apache.hadoop.hbase.regionserver; import java.lang.reflect.Method; import java.util.HashMap; import java.util.Map; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.ipc.PriorityFunction; +import org.apache.hadoop.hbase.ipc.QosPriority; +import org.apache.hadoop.hbase.security.User; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.hbase.ipc.PriorityFunction; -import org.apache.hadoop.hbase.ipc.QosPriority; + +import org.apache.hbase.thirdparty.com.google.protobuf.Message; +import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat; + import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest; @@ -39,11 +43,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanReques import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; -import org.apache.hbase.thirdparty.com.google.protobuf.Message; -import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat; -import org.apache.hadoop.hbase.security.User; - /** * Reads special method annotations and table names to figure a priority for use by QoS facility in * ipc; e.g: rpcs to hbase:meta get priority. @@ -275,7 +274,6 @@ public class AnnotationReadingPriorityFunction implements PriorityFunction { return 0; } - @VisibleForTesting void setRegionServer(final HRegionServer hrs) { this.rpcServices = hrs.getRSRpcServices(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellSet.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellSet.java index d74655d1b7a..e1cfd15ed19 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellSet.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellSet.java @@ -18,8 +18,6 @@ */ package org.apache.hadoop.hbase.regionserver; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import java.util.Collection; import java.util.Comparator; import java.util.Iterator; @@ -30,6 +28,7 @@ import java.util.concurrent.ConcurrentSkipListMap; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.HConstants; import org.apache.yetus.audience.InterfaceAudience; /** @@ -62,13 +61,11 @@ public class CellSet implements NavigableSet { this.numUniqueKeys = numUniqueKeys; } - @VisibleForTesting CellSet(final NavigableMap m) { this.delegatee = m; this.numUniqueKeys = UNKNOWN_NUM_UNIQUES; } - @VisibleForTesting NavigableMap getDelegatee() { return delegatee; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java index 136efeec201..1023890dac2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java @@ -19,11 +19,9 @@ package org.apache.hadoop.hbase.regionserver; import java.nio.ByteBuffer; import java.util.concurrent.atomic.AtomicInteger; - -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; /** @@ -176,7 +174,6 @@ public abstract class Chunk { + (data.capacity() - nextFreeOffset.get()); } - @VisibleForTesting int getNextFreeOffset() { return this.nextFreeOffset.get(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java index d9f32716756..5245ac2853e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java @@ -28,15 +28,13 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.LongAdder; - -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.hadoop.hbase.regionserver.HeapMemoryManager.HeapMemoryTuneObserver; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.util.StringUtils; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; /** @@ -75,15 +73,12 @@ public class ChunkCreator { private Map chunkIdMap = new ConcurrentHashMap(); private final boolean offheap; - @VisibleForTesting static ChunkCreator instance; - @VisibleForTesting static boolean chunkPoolDisabled = false; private MemStoreChunkPool dataChunksPool; private final int chunkSize; private MemStoreChunkPool indexChunksPool; - @VisibleForTesting ChunkCreator(int chunkSize, boolean offheap, long globalMemStoreSize, float poolSizePercentage, float initialCountPercentage, HeapMemoryManager heapMemoryManager, float indexChunkSizePercentage) { @@ -93,7 +88,6 @@ public class ChunkCreator { initialCountPercentage, heapMemoryManager); } - @VisibleForTesting private void initializePools(int chunkSize, long globalMemStoreSize, float poolSizePercentage, float indexChunkSizePercentage, float initialCountPercentage, @@ -122,7 +116,6 @@ public class ChunkCreator { */ @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "LI_LAZY_INIT_STATIC", justification = "Method is called by single thread at the starting of RS") - @VisibleForTesting public static ChunkCreator initialize(int chunkSize, boolean offheap, long globalMemStoreSize, float poolSizePercentage, float initialCountPercentage, HeapMemoryManager heapMemoryManager, @@ -135,7 +128,6 @@ public class ChunkCreator { return instance; } - @VisibleForTesting public static ChunkCreator getInstance() { return instance; } @@ -280,8 +272,7 @@ public class ChunkCreator { return createChunk(true, chunkIndexType, chunkSize); } - @VisibleForTesting - // Used to translate the ChunkID into a chunk ref + // Used to translate the ChunkID into a chunk ref Chunk getChunk(int id) { // can return null if chunk was never mapped return chunkIdMap.get(id); @@ -299,14 +290,12 @@ public class ChunkCreator { return this.chunkIdMap.remove(chunkId); } - @VisibleForTesting - // the chunks in the chunkIdMap may already be released so we shouldn't relay - // on this counting for strong correctness. This method is used only in testing. + // the chunks in the chunkIdMap may already be released so we shouldn't relay + // on this counting for strong correctness. This method is used only in testing. int numberOfMappedChunks() { return this.chunkIdMap.size(); } - @VisibleForTesting void clearChunkIds() { this.chunkIdMap.clear(); } @@ -471,7 +460,6 @@ public class ChunkCreator { } } - @VisibleForTesting static void clearDisableFlag() { chunkPoolDisabled = false; } @@ -507,12 +495,10 @@ public class ChunkCreator { return memStoreChunkPool; } - @VisibleForTesting int getMaxCount() { return getMaxCount(ChunkType.DATA_CHUNK); } - @VisibleForTesting int getMaxCount(ChunkType chunkType) { switch (chunkType) { case INDEX_CHUNK: @@ -533,12 +519,10 @@ public class ChunkCreator { return 0; } - @VisibleForTesting int getPoolSize() { return getPoolSize(ChunkType.DATA_CHUNK); } - @VisibleForTesting int getPoolSize(ChunkType chunkType) { switch (chunkType) { case INDEX_CHUNK: @@ -558,7 +542,6 @@ public class ChunkCreator { return 0; } - @VisibleForTesting boolean isChunkInPool(int chunkId) { Chunk c = getChunk(chunkId); if (c==null) { @@ -577,7 +560,6 @@ public class ChunkCreator { /* * Only used in testing */ - @VisibleForTesting void clearChunksInPool() { if (dataChunksPool != null) { dataChunksPool.reclaimedChunks.clear(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java index 7516c54e625..441b18b3302 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver; import static org.apache.hadoop.hbase.regionserver.Store.NO_PRIORITY; import static org.apache.hadoop.hbase.regionserver.Store.PRIORITY_USER; + import java.io.IOException; import java.io.PrintWriter; import java.io.StringWriter; @@ -53,7 +54,7 @@ import org.apache.hadoop.util.StringUtils; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -779,12 +780,10 @@ public class CompactSplit implements CompactionRequester, PropagatingConfigurati // No children to register } - @VisibleForTesting public ThroughputController getCompactionThroughputController() { return compactionThroughputController; } - @VisibleForTesting /** * Shutdown the long compaction thread pool. * Should only be used in unit test to prevent long compaction thread pool from stealing job diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java index 2f24135ccc7..2be04c22729 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java @@ -26,7 +26,6 @@ import org.apache.hadoop.hbase.executor.EventType; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; /** * A chore service that periodically cleans up the compacted files when there are no active readers @@ -38,7 +37,6 @@ public class CompactedHFilesDischarger extends ScheduledChore { private static final Logger LOG = LoggerFactory.getLogger(CompactedHFilesDischarger.class); private RegionServerServices regionServerServices; // Default is to use executor - @VisibleForTesting private boolean useExecutor = true; /** @@ -59,7 +57,6 @@ public class CompactedHFilesDischarger extends ScheduledChore { * @param regionServerServices the region server that starts this chore * @param useExecutor true if to use the region server's executor service, false otherwise */ - @VisibleForTesting public CompactedHFilesDischarger(final int period, final Stoppable stopper, final RegionServerServices regionServerServices, boolean useExecutor) { // Need to add the config classes @@ -73,7 +70,6 @@ public class CompactedHFilesDischarger extends ScheduledChore { * cleanup. Use this method to set no-executor before you call run. * @return The old setting for useExecutor */ - @VisibleForTesting boolean setUseExecutor(final boolean useExecutor) { boolean oldSetting = this.useExecutor; this.useExecutor = useExecutor; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java index 76292736558..abe9cf8c23c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java @@ -18,9 +18,6 @@ */ package org.apache.hadoop.hbase.regionserver; -import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException; -import org.apache.hadoop.util.StringUtils; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -32,13 +29,15 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MemoryCompactionPolicy; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.util.StringUtils; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A memstore implementation which supports in-memory compaction. @@ -79,7 +78,6 @@ public class CompactingMemStore extends AbstractMemStore { // inWalReplay is true while we are synchronously replaying the edits from WAL private boolean inWalReplay = false; - @VisibleForTesting protected final AtomicBoolean allowCompaction = new AtomicBoolean(true); private boolean compositeSnapshot = true; @@ -128,7 +126,6 @@ public class CompactingMemStore extends AbstractMemStore { (this.compactor == null? "NULL": this.compactor.toString())); } - @VisibleForTesting protected MemStoreCompactor createMemStoreCompactor(MemoryCompactionPolicy compactionPolicy) throws IllegalArgumentIOException { return new MemStoreCompactor(this, compactionPolicy); @@ -334,7 +331,6 @@ public class CompactingMemStore extends AbstractMemStore { } // the getSegments() method is used for tests only - @VisibleForTesting @Override protected List getSegments() { List pipelineList = pipeline.getSegments(); @@ -367,7 +363,6 @@ public class CompactingMemStore extends AbstractMemStore { } // setter is used only for testability - @VisibleForTesting void setIndexType(IndexType type) { indexType = type; // Because this functionality is for testing only and tests are setting in-memory flush size @@ -413,7 +408,6 @@ public class CompactingMemStore extends AbstractMemStore { return list; } - @VisibleForTesting protected List createList(int capacity) { return new ArrayList<>(capacity); } @@ -451,7 +445,6 @@ public class CompactingMemStore extends AbstractMemStore { // externally visible only for tests // when invoked directly from tests it must be verified that the caller doesn't hold updatesLock, // otherwise there is a deadlock - @VisibleForTesting void flushInMemory() { MutableSegment currActive = getActive(); if(currActive.setInMemoryFlushed()) { @@ -499,7 +492,6 @@ public class CompactingMemStore extends AbstractMemStore { return getRegionServices().getInMemoryCompactionPool(); } - @VisibleForTesting protected boolean shouldFlushInMemory(MutableSegment currActive, Cell cellToAdd, MemStoreSizing memstoreSizing) { long cellSize = MutableSegment.getCellLength(cellToAdd); @@ -596,7 +588,6 @@ public class CompactingMemStore extends AbstractMemStore { } } - @VisibleForTesting boolean isMemStoreFlushingInMemory() { return inMemoryCompactionInProgress.get(); } @@ -619,7 +610,6 @@ public class CompactingMemStore extends AbstractMemStore { return lowest; } - @VisibleForTesting long getInmemoryFlushSize() { return inmemoryFlushSize; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompositeImmutableSegment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompositeImmutableSegment.java index 19647faa9de..53ef82d5695 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompositeImmutableSegment.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompositeImmutableSegment.java @@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.io.TimeRange; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; /** * The CompositeImmutableSegments is created as a collection of ImmutableSegments and supports @@ -52,7 +51,6 @@ public class CompositeImmutableSegment extends ImmutableSegment { } } - @VisibleForTesting @Override public List getAllSegments() { return new ArrayList<>(segments); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ConstantSizeRegionSplitPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ConstantSizeRegionSplitPolicy.java index 5ffd96021fe..1f22dc4d2b2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ConstantSizeRegionSplitPolicy.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ConstantSizeRegionSplitPolicy.java @@ -17,8 +17,6 @@ */ package org.apache.hadoop.hbase.regionserver; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - import java.util.Random; import org.apache.hadoop.conf.Configuration; @@ -86,7 +84,7 @@ public class ConstantSizeRegionSplitPolicy extends RegionSplitPolicy { return desiredMaxFileSize; } - @VisibleForTesting + @InterfaceAudience.Private public boolean positiveJitterRate() { return this.jitterRate > 0; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index c30f586a8ff..fc3c6234109 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.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @@ -319,7 +318,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // Track data size in all memstores private final MemStoreSizing memStoreSizing = new ThreadSafeMemStoreSizing(); - @VisibleForTesting RegionServicesForStores regionServicesForStores; // Debug possible data loss due to WAL off @@ -581,7 +579,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } /** A result object from prepare flush cache stage */ - @VisibleForTesting static class PrepareFlushResult { final FlushResultImpl result; // indicating a failure result from prepare final TreeMap storeFlushCtxs; @@ -746,7 +743,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi * @deprecated Use other constructors. */ @Deprecated - @VisibleForTesting public HRegion(final Path tableDir, final WAL wal, final FileSystem fs, final Configuration confParam, final RegionInfo regionInfo, final TableDescriptor htd, final RegionServerServices rsServices) { @@ -951,7 +947,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi * @return What the next sequence (edit) id should be. * @throws IOException e */ - @VisibleForTesting long initialize(final CancelableProgressable reporter) throws IOException { //Refuse to open the region if there is no column family in the table @@ -1230,7 +1225,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi return allStoreFiles; } - @VisibleForTesting protected void writeRegionOpenMarker(WAL wal, long openSeqId) throws IOException { Map> storeFiles = getStoreFiles(); RegionEventDescriptor regionOpenDesc = ProtobufUtil.toRegionEventDescriptor( @@ -1514,7 +1508,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } } - @VisibleForTesting public MultiVersionConcurrencyControl getMVCC() { return mvcc; } @@ -1637,7 +1630,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi /** * Exposed for some very specific unit tests. */ - @VisibleForTesting public void setClosing(boolean closing) { this.closing.set(closing); } @@ -1647,7 +1639,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi * Instead of blocking, the {@link HRegion#doClose} will throw exception if you set the timeout. * @param timeoutForWriteLock the second time to wait for the write lock in {@link HRegion#doClose} */ - @VisibleForTesting public void setTimeoutForWriteLock(long timeoutForWriteLock) { assert timeoutForWriteLock >= 0; this.timeoutForWriteLock = timeoutForWriteLock; @@ -2045,7 +2036,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi return this.htableDescriptor; } - @VisibleForTesting public void setTableDescriptor(TableDescriptor desc) { htableDescriptor = desc; } @@ -2062,7 +2052,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi /** * Only used for unit test which doesn't start region server. */ - @VisibleForTesting public void setBlockCache(BlockCache blockCache) { this.blockCache = blockCache; } @@ -2074,7 +2063,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi /** * Only used for unit test which doesn't start region server. */ - @VisibleForTesting public void setMobFileCache(MobFileCache mobFileCache) { this.mobFileCache = mobFileCache; } @@ -2082,7 +2070,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi /** * @return split policy for this region. */ - @VisibleForTesting RegionSplitPolicy getSplitPolicy() { return this.splitPolicy; } @@ -2126,7 +2113,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi * @return the Region directory under WALRootDirectory * @throws IOException if there is an error getting WALRootDir */ - @VisibleForTesting public Path getWALRegionDir() throws IOException { if (regionDir == null) { regionDir = CommonFSUtils.getWALRegionDir(conf, getRegionInfo().getTable(), @@ -2235,7 +2221,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi *

* It is used by utilities and testing */ - @VisibleForTesting public void compactStores() throws IOException { for (HStore s : stores.values()) { Optional compaction = s.requestCompaction(); @@ -2250,7 +2235,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi *

* It is used by utilities and testing */ - @VisibleForTesting void compactStore(byte[] family, ThroughputController throughputController) throws IOException { HStore s = getStore(family); Optional compaction = s.requestCompaction(); @@ -3131,7 +3115,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi * @return Next sequence number unassociated with any actual edit. * @throws IOException */ - @VisibleForTesting protected long getNextSequenceId(final WAL wal) throws IOException { WriteEntry we = mvcc.begin(); mvcc.completeAndWait(we); @@ -5306,7 +5289,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi * @return the sequence id of the last edit added to this region out of the * recovered edits log or minSeqId if nothing added from editlogs. */ - @VisibleForTesting long replayRecoveredEditsIfAny(Map maxSeqIdInStores, final CancelableProgressable reporter, final MonitoredTask status) throws IOException { long minSeqIdForTheRegion = -1; @@ -5772,7 +5754,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi * the store memstores, only if the memstores do not have a higher seqId from an earlier wal * edit (because the events may be coming out of order). */ - @VisibleForTesting PrepareFlushResult replayWALFlushStartMarker(FlushDescriptor flush) throws IOException { long flushSeqId = flush.getFlushSequenceNumber(); @@ -5883,7 +5864,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi return null; } - @VisibleForTesting @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY", justification="Intentional; post memstore flush") void replayWALFlushCommitMarker(FlushDescriptor flush) throws IOException { @@ -6171,7 +6151,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } } - @VisibleForTesting PrepareFlushResult getPrepareFlushResult() { return prepareFlushResult; } @@ -6535,7 +6514,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi * @param s Store to add edit too. * @param cell Cell to add. */ - @VisibleForTesting protected void restoreEdit(HStore s, Cell cell, MemStoreSizing memstoreAccounting) { s.add(cell, memstoreAccounting); } @@ -6738,7 +6716,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } } - @VisibleForTesting public int getReadLockCount() { return lock.getReadLockCount(); } @@ -6747,7 +6724,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi return lockedRows; } - @VisibleForTesting class RowLockContext { private final HashedBytes row; final ReadWriteLock readWriteLock = new ReentrantReadWriteLock(true); @@ -6824,7 +6800,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi return lock; } - @VisibleForTesting public RowLockContext getContext() { return context; } @@ -8114,7 +8089,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi return r.openHRegion(reporter); } - @VisibleForTesting public NavigableMap getReplicationScope() { return this.replicationScope; } @@ -8890,7 +8864,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } /** @param coprocessorHost the new coprocessor host */ - @VisibleForTesting public void setCoprocessorHost(final RegionCoprocessorHost coprocessorHost) { this.coprocessorHost = coprocessorHost; } @@ -9190,7 +9163,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi flushesQueued.increment(); } - @VisibleForTesting public long getReadPoint() { return getReadPoint(IsolationLevel.READ_COMMITTED); } @@ -9390,12 +9362,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } } - @VisibleForTesting public void setReadRequestsCount(long readRequestsCount) { this.readRequestsCount.add(readRequestsCount); } - @VisibleForTesting public void setWriteRequestsCount(long writeRequestsCount) { this.writeRequestsCount.add(writeRequestsCount); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java index d5ef30ecc79..73234f161c4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java @@ -54,7 +54,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; /** @@ -75,7 +74,7 @@ public class HRegionFileSystem { public static final String REGION_SPLITS_DIR = ".splits"; /** Temporary subdirectory of the region directory used for compaction output. */ - @VisibleForTesting static final String REGION_TEMP_DIR = ".tmp"; + static final String REGION_TEMP_DIR = ".tmp"; private final RegionInfo regionInfo; //regionInfo for interacting with FS (getting encodedName, etc) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index c4261dc8e35..1c2f68c04ea 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.AbstractFSWALProvider; import org.apache.hadoop.hbase.wal.NettyAsyncFSWALConfigHelper; import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.wal.WALFactory; -import org.apache.hadoop.hbase.wal.WALProvider; import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker; import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; @@ -192,7 +191,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import sun.misc.Signal; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; import org.apache.hbase.thirdparty.com.google.common.base.Throwables; import org.apache.hbase.thirdparty.com.google.common.cache.Cache; @@ -253,7 +251,7 @@ public class HRegionServer extends Thread implements /** * For testing only! Set to true to skip notifying region assignment to master . */ - @VisibleForTesting + @InterfaceAudience.Private @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="MS_SHOULD_BE_FINAL") public static boolean TEST_SKIP_REPORTING_TRANSITION = false; @@ -545,7 +543,7 @@ public class HRegionServer extends Thread implements */ protected final ConfigurationManager configurationManager; - @VisibleForTesting + @InterfaceAudience.Private CompactedHFilesDischarger compactedFileDischarger; private volatile ThroughputController flushThroughputController; @@ -1244,7 +1242,7 @@ public class HRegionServer extends Thread implements return writeCount; } - @VisibleForTesting + @InterfaceAudience.Private protected void tryRegionServerReport(long reportStartTime, long reportEndTime) throws IOException { RegionServerStatusService.BlockingInterface rss = rssStub; @@ -2521,7 +2519,7 @@ public class HRegionServer extends Thread implements return rpcServices.rpcServer; } - @VisibleForTesting + @InterfaceAudience.Private public RSRpcServices getRSRpcServices() { return rpcServices; } @@ -2611,7 +2609,7 @@ public class HRegionServer extends Thread implements * logs but it does close socket in case want to bring up server on old * hostname+port immediately. */ - @VisibleForTesting + @InterfaceAudience.Private protected void kill() { this.killed = true; abort("Simulated kill"); @@ -2720,7 +2718,7 @@ public class HRegionServer extends Thread implements * @param refresh If true then master address will be read from ZK, otherwise use cached data * @return master + port, or null if server has been stopped */ - @VisibleForTesting + @InterfaceAudience.Private protected synchronized ServerName createRegionServerStatusStub(boolean refresh) { if (rssStub != null) { return masterAddressTracker.getMasterAddress(); @@ -3559,12 +3557,12 @@ public class HRegionServer extends Thread implements movedRegionInfoCache.invalidate(encodedName); } - @VisibleForTesting + @InterfaceAudience.Private public MovedRegionInfo getMovedRegion(String encodedRegionName) { return movedRegionInfoCache.getIfPresent(encodedRegionName); } - @VisibleForTesting + @InterfaceAudience.Private public int movedRegionCacheExpiredTime() { return TIMEOUT_REGION_MOVED; } @@ -3660,7 +3658,7 @@ public class HRegionServer extends Thread implements /** * @return : Returns the ConfigurationManager object for testing purposes. */ - @VisibleForTesting + @InterfaceAudience.Private ConfigurationManager getConfigurationManager() { return configurationManager; } @@ -3724,7 +3722,7 @@ public class HRegionServer extends Thread implements * For testing * @return whether all wal roll request finished for this regionserver */ - @VisibleForTesting + @InterfaceAudience.Private public boolean walRollRequestFinished() { return this.walRoller.walRollFinished(); } @@ -3929,7 +3927,7 @@ public class HRegionServer extends Thread implements 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 a05d4a6d31e..3a71c230beb 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.throttle.ThroughputController; 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.hadoop.util.StringUtils.TraditionalBinaryPrefix; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableCollection; import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; @@ -119,6 +117,10 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Maps; import org.apache.hbase.thirdparty.com.google.common.collect.Sets; import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils; import org.apache.hbase.thirdparty.org.apache.commons.collections4.IterableUtils; + +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor; + /** * A Store holds a column family in a Region. Its a memstore and a set of zero * or more StoreFiles, which stretch backwards over time. @@ -725,7 +727,6 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, completeCompaction(toBeRemovedStoreFiles); } - @VisibleForTesting protected HStoreFile createStoreFileAndReader(final Path p) throws IOException { StoreFileInfo info = new StoreFileInfo(conf, this.getFileSystem(), p, isPrimaryReplicaStore()); @@ -1538,7 +1539,6 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, } } - @VisibleForTesting protected List doCompaction(CompactionRequestImpl cr, Collection filesToCompact, User user, long compactionStartTime, List newFiles) throws IOException { @@ -1633,7 +1633,6 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, this.region.getRegionInfo(), compactionDescriptor, this.region.getMVCC()); } - @VisibleForTesting void replaceStoreFiles(Collection compactedFiles, Collection result) throws IOException { this.lock.writeLock().lock(); @@ -1794,7 +1793,6 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, * but instead makes a compaction candidate list by itself. * @param N Number of files. */ - @VisibleForTesting public void compactRecentForTestingAssumingDefaultPolicy(int N) throws IOException { List filesToCompact; boolean isMajor; @@ -2069,7 +2067,6 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, * Update counts. * @param compactedFiles list of files that were compacted */ - @VisibleForTesting protected void completeCompaction(Collection compactedFiles) // Rename this method! TODO. throws IOException { @@ -2561,7 +2558,6 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, * Used for tests. * @return cache configuration for this Store. */ - @VisibleForTesting public CacheConfig getCacheConfig() { return this.cacheConf; } @@ -2641,7 +2637,6 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, * Returns the StoreEngine that is backing this concrete implementation of Store. * @return Returns the {@link StoreEngine} object used internally inside this HStore object. */ - @VisibleForTesting public StoreEngine getStoreEngine() { return this.storeEngine; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java index 5e0bf2a43e9..7a7468973af 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.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; /** @@ -319,7 +319,6 @@ public class HStoreFile implements StoreFile { return compactedAway; } - @VisibleForTesting public int getRefCount() { return fileInfo.refCount.get(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java index ea5586c4af9..1f831eefee6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java @@ -39,8 +39,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * Manages tuning of Heap memory using HeapMemoryTuner. Most part of the heap memory is * split between Memstores and BlockCache. This manager helps in tuning sizes of both these @@ -106,7 +104,6 @@ public class HeapMemoryManager { private List tuneObservers = new ArrayList<>(); - @VisibleForTesting HeapMemoryManager(BlockCache blockCache, FlushRequester memStoreFlusher, Server server, RegionServerAccounting regionServerAccounting) { Configuration conf = server.getConfiguration(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java index f28e28e31cc..2fef5d106f0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java @@ -19,8 +19,6 @@ package org.apache.hadoop.hbase.regionserver; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - import java.io.IOException; import java.util.ArrayList; import java.util.Comparator; @@ -29,10 +27,10 @@ import java.util.PriorityQueue; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState; /** * Implements a heap merge across any number of KeyValueScanners. @@ -419,8 +417,6 @@ public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner return this.heap; } - - @VisibleForTesting KeyValueScanner getCurrentForTesting() { return current; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java index cbab595517f..be73bbaf8a3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java @@ -27,8 +27,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * Runs periodically to determine if the WAL should be rolled. * @@ -39,7 +37,6 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti * TODO: change to a pool of threads */ @InterfaceAudience.Private -@VisibleForTesting public class LogRoller extends AbstractWALRoller { private static final Logger LOG = LoggerFactory.getLogger(LogRoller.class); @@ -64,7 +61,6 @@ public class LogRoller extends AbstractWALRoller { requester.requestFlush(r, families, FlushLifeCycleTracker.DUMMY); } - @VisibleForTesting Map getWalNeedsRoll() { return this.wals; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactor.java index 2dafceee2e8..a0313444a53 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactor.java @@ -32,8 +32,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * The ongoing MemStore Compaction manager, dispatches a solo running compaction and interrupts * the compaction if requested. The compaction is interrupted and stopped by CompactingMemStore, @@ -242,7 +240,6 @@ public class MemStoreCompactor { return result; } - @VisibleForTesting void initiateCompactionStrategy(MemoryCompactionPolicy compType, Configuration configuration, String cfName) throws IllegalArgumentIOException { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java index 7951c72b0da..f5fccf4d2a8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java @@ -27,7 +27,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.ReentrantLock; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ByteBufferExtendedCell; import org.apache.hadoop.hbase.Cell; @@ -36,8 +35,9 @@ import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; + /** * A memstore-local allocation buffer. *

@@ -73,7 +73,6 @@ public class MemStoreLABImpl implements MemStoreLAB { private ReentrantLock lock = new ReentrantLock(); // A set of chunks contained by this memstore LAB - @VisibleForTesting Set chunks = new ConcurrentSkipListSet(); private final int dataChunkSize; private final int maxAlloc; @@ -270,7 +269,6 @@ public class MemStoreLABImpl implements MemStoreLAB { } } - @VisibleForTesting int getOpenScannerCount() { return this.openScannerCount.get(); } @@ -397,12 +395,10 @@ public class MemStoreLABImpl implements MemStoreLAB { return this.chunkCreator.isOffheap(); } - @VisibleForTesting Chunk getCurrentChunk() { return currChunk.get(); } - @VisibleForTesting BlockingQueue getPooledChunks() { BlockingQueue pooledChunks = new LinkedBlockingQueue<>(); for (Integer id : this.chunks) { @@ -414,7 +410,7 @@ public class MemStoreLABImpl implements MemStoreLAB { return pooledChunks; } - @VisibleForTesting Integer getNumOfChunksReturnedToPool() { + Integer getNumOfChunksReturnedToPool() { int i = 0; for (Integer id : this.chunks) { if (chunkCreator.isChunkInPool(id)) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java index e37a2722c9f..6bbb620e140 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java @@ -27,8 +27,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** *

* This class is for maintaining the various regionserver statistics @@ -95,12 +93,10 @@ public class MetricsRegionServer { return null; } - @VisibleForTesting public MetricsRegionServerSource getMetricsSource() { return serverSource; } - @VisibleForTesting public MetricsUserAggregate getMetricsUserAggregate() { return userAggregate; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiVersionConcurrencyControl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiVersionConcurrencyControl.java index 2b350e686f9..d821eecf6f3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiVersionConcurrencyControl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiVersionConcurrencyControl.java @@ -18,19 +18,16 @@ */ package org.apache.hadoop.hbase.regionserver; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; -import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects; -import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects.ToStringHelper; - import java.util.LinkedList; import java.util.concurrent.atomic.AtomicLong; - +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.ClassSize; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.ClassSize; +import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects; +import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects.ToStringHelper; /** * Manages the read/write consistency. This provides an interface for readers to determine what @@ -251,7 +248,6 @@ public class MultiVersionConcurrencyControl { } } - @VisibleForTesting @Override public String toString() { ToStringHelper helper = MoreObjects.toStringHelper(this).add("readPoint", readPoint) @@ -266,7 +262,6 @@ public class MultiVersionConcurrencyControl { return readPoint.get(); } - @VisibleForTesting public long getWritePoint() { return writePoint.get(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableSegment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableSegment.java index 2c1e859fbd4..6e813433fac 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableSegment.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableSegment.java @@ -28,10 +28,8 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.KeyValue; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.util.ClassSize; - -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.yetus.audience.InterfaceAudience; /** * A mutable segment in memstore, specifically the active segment. @@ -124,7 +122,6 @@ public class MutableSegment extends Segment { * Returns the first cell in the segment * @return the first cell in the segment */ - @VisibleForTesting Cell first() { return this.getCellSet().first(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java index 9efdaa993d6..5d9327bff54 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.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.cache.Cache; import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @@ -328,7 +327,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler, final RpcServerInterface rpcServer; final InetSocketAddress isa; - @VisibleForTesting protected final HRegionServer regionServer; private final long maxScannerResultSize; @@ -1236,7 +1234,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler, } } - @VisibleForTesting public int getScannersCount() { return scanners.size(); } @@ -1374,7 +1371,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler, * @throws IOException if the specifier is not null, * but failed to find the region */ - @VisibleForTesting public HRegion getRegion( final RegionSpecifier regionSpecifier) throws IOException { return regionServer.getRegion(regionSpecifier.getValue().toByteArray()); @@ -1401,12 +1397,10 @@ public class RSRpcServices implements HBaseRPCErrorHandler, return regions; } - @VisibleForTesting public PriorityFunction getPriority() { return priority; } - @VisibleForTesting public Configuration getConfiguration() { return regionServer.getConfiguration(); } @@ -3912,7 +3906,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler, throw new ServiceException("Invalid request params"); } - @VisibleForTesting public RpcScheduler getRpcScheduler() { return rpcServer.getScheduler(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java index 36392d7ef73..06795a58545 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java @@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.io.ByteBuffAllocator; import org.apache.hadoop.hbase.wal.WAL; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; /** @@ -113,7 +112,6 @@ public class RegionServicesForStores { return region.getTableDescriptor().getColumnFamilyCount(); } - @VisibleForTesting long getMemStoreSize() { return region.getMemStoreDataSize(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanInfo.java index 7d6161849ad..4e2066601c5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanInfo.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanInfo.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hbase.regionserver; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; @@ -26,15 +25,13 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeepDeletedCells; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; - -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.yetus.audience.InterfaceAudience; /** * Immutable information for scans over a store. */ // Has to be public for PartitionedMobCompactor to access; ditto on tests making use of a few of // the accessors below. Shutdown access. TODO -@VisibleForTesting @InterfaceAudience.Private public class ScanInfo { private byte[] family; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java index 15d87101a8f..e8a9154d410 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.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest; @@ -166,12 +164,10 @@ public class SecureBulkLoadManager { private Consumer fsCreatedListener; - @VisibleForTesting void setFsCreatedListener(Consumer fsCreatedListener) { this.fsCreatedListener = fsCreatedListener; } - private void incrementUgiReference(UserGroupInformation ugi) { // if we haven't seen this ugi before, make a new counter ugiReferenceCounter.compute(ugi, (key, value) -> { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java index b1c92a4e8d5..b0763aa3835 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java @@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; /** * This is an abstraction of a segment maintained in a memstore, e.g., the active @@ -181,7 +180,6 @@ public abstract class Segment implements MemStoreSizing { /** * Get cell length after serialized in {@link KeyValue} */ - @VisibleForTesting static int getCellLength(Cell cell) { return cell.getSerializedSize(); } @@ -414,7 +412,6 @@ public abstract class Segment implements MemStoreSizing { return getCellSet().tailSet(firstCell); } - @VisibleForTesting MemStoreLAB getMemStoreLAB() { return memStoreLAB; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ServerNonceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ServerNonceManager.java index 0b1d251d989..e0ea974b194 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ServerNonceManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ServerNonceManager.java @@ -27,13 +27,11 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ScheduledChore; import org.apache.hadoop.hbase.Stoppable; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.NonceKey; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.util.NonceKey; - -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; /** * Implementation of nonce manager that stores nonces in a hash map and cleans them up after @@ -134,7 +132,6 @@ public class ServerNonceManager { } } - @VisibleForTesting public void setConflictWaitIterationMs(int conflictWaitIterationMs) { this.conflictWaitIterationMs = conflictWaitIterationMs; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java index 90e705c7186..ed3a73fbcfa 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.hadoop.hbase.wal.WALSplitter; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; /** * This worker is spawned in every regionserver, including master. The Worker waits for log @@ -289,7 +288,6 @@ public class SplitLogWorker implements Runnable { * Returns the number of tasks processed by coordination. * This method is used by tests only */ - @VisibleForTesting public int getTaskReadySeq() { return coordination.getTaskReadySeq(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java index f92a4d38659..7550511a356 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java @@ -54,7 +54,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; /** * Reader for a StoreFile. @@ -115,7 +114,7 @@ public class StoreFileReader { /** * ONLY USE DEFAULT CONSTRUCTOR FOR UNIT TESTS */ - @VisibleForTesting + @InterfaceAudience.Private StoreFileReader() { this.refCount = new AtomicInteger(0); this.reader = null; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java index 8c48b124645..1744a7f06ab 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java @@ -25,7 +25,6 @@ import java.util.List; import java.util.NavigableSet; import java.util.concurrent.CountDownLatch; import java.util.concurrent.locks.ReentrantLock; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellUtil; @@ -46,11 +45,9 @@ import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher; import org.apache.hadoop.hbase.regionserver.querymatcher.UserScanQueryMatcher; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.yetus.audience.InterfaceAudience; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils; @@ -93,9 +90,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner private final int minVersions; private final long maxRowSize; private final long cellsPerHeartbeatCheck; - @VisibleForTesting long memstoreOnlyReads; - @VisibleForTesting long mixedReads; // 1) Collects all the KVHeap that are eagerly getting closed during the @@ -155,7 +150,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner // Since CompactingMemstore is now default, we get three memstore scanners from a flush private final List memStoreScannersAfterFlush = new ArrayList<>(3); // The current list of scanners - @VisibleForTesting final List currentScanners = new ArrayList<>(); // flush update lock private final ReentrantLock flushLock = new ReentrantLock(); @@ -344,7 +338,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner } // Used to instantiate a scanner for user scan in test - @VisibleForTesting StoreScanner(Scan scan, ScanInfo scanInfo, NavigableSet columns, List scanners, ScanType scanType) throws IOException { // 0 is passed as readpoint because the test bypasses Store @@ -361,7 +354,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner } // Used to instantiate a scanner for user scan in test - @VisibleForTesting StoreScanner(Scan scan, ScanInfo scanInfo, NavigableSet columns, List scanners) throws IOException { // 0 is passed as readpoint because the test bypasses Store @@ -373,7 +365,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner } // Used to instantiate a scanner for compaction in test - @VisibleForTesting StoreScanner(ScanInfo scanInfo, int maxVersions, ScanType scanType, List scanners) throws IOException { // 0 is passed as readpoint because the test bypasses Store @@ -384,7 +375,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner seekAllScanner(scanInfo, scanners); } - @VisibleForTesting boolean isScanUsePread() { return this.scanUsePread; } @@ -427,7 +417,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner } } - @VisibleForTesting protected void resetKVHeap(List scanners, CellComparator comparator) throws IOException { // Combine all seeked scanners with a heap @@ -444,7 +433,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner *

* Will be overridden by testcase so declared as protected. */ - @VisibleForTesting protected List selectScannersFrom(HStore store, List allScanners) { boolean memOnly; @@ -870,7 +858,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner * @param cell current cell * @return true means skip to next row, false means not */ - @VisibleForTesting protected boolean trySkipToNextRow(Cell cell) throws IOException { Cell nextCell = null; // used to guard against a changed next indexed key by doing a identity comparison @@ -896,7 +883,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner * @param cell current cell * @return true means skip to next column, false means not */ - @VisibleForTesting protected boolean trySkipToNextColumn(Cell cell) throws IOException { Cell nextCell = null; // used to guard against a changed next indexed key by doing a identity comparison @@ -1096,7 +1082,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner return heap.reseek(kv); } - @VisibleForTesting void trySwitchToStreamRead() { if (readType != Scan.ReadType.DEFAULT || !scanUsePread || closing || heap.peek() == null || bytesRead < preadMaxBytes) { @@ -1208,7 +1193,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner * Used in testing. * @return all scanners in no particular order */ - @VisibleForTesting List getAllScannersForTesting() { List allScanners = new ArrayList<>(); KeyValueScanner current = heap.getCurrentForTesting(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java index 7beec5e2e8a..1560aef5f6b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java @@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; /** * Stripe implementation of StoreFlusher. Flushes files either into L0 file w/o metadata, or @@ -119,7 +118,6 @@ public class StripeStoreFlusher extends StoreFlusher { this.comparator = comparator; } - @VisibleForTesting public StripeMultiFileWriter createWriter() throws IOException { StripeMultiFileWriter writer = new StripeMultiFileWriter.SizeMultiWriter(comparator, 1, Long.MAX_VALUE, OPEN_KEY, OPEN_KEY); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/TimeRangeTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/TimeRangeTracker.java index 37923ad4d46..fdf9db273a6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/TimeRangeTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/TimeRangeTracker.java @@ -24,14 +24,13 @@ import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; import java.util.concurrent.atomic.AtomicLong; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.io.TimeRange; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; @@ -243,7 +242,6 @@ public abstract class TimeRangeTracker { return 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 @@ public abstract class TimeRangeTracker { } } - @VisibleForTesting //In order to estimate the heap size, this inner class need to be accessible to TestHeapSize. public static class SyncTimeRangeTracker extends TimeRangeTracker { private final AtomicLong minimumTimestamp = new AtomicLong(INITIAL_MIN_TIMESTAMP); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/VersionedSegmentsList.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/VersionedSegmentsList.java index 2a684da0131..d5be356f93f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/VersionedSegmentsList.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/VersionedSegmentsList.java @@ -21,8 +21,6 @@ package org.apache.hadoop.hbase.regionserver; import java.util.List; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * A list of segment managers coupled with the version of the memstore (version at the time it was * created). @@ -65,7 +63,6 @@ public class VersionedSegmentsList { } // Estimates fraction of unique keys - @VisibleForTesting double getEstimatedUniquesFrac() { int segmentCells = 0; int maxCells = 0; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CurrentHourProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CurrentHourProvider.java index b68363498e4..1f5a8208f0a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CurrentHourProvider.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CurrentHourProvider.java @@ -22,7 +22,6 @@ import java.util.GregorianCalendar; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; @InterfaceAudience.Private public class CurrentHourProvider { @@ -38,7 +37,6 @@ public class CurrentHourProvider { } } - @VisibleForTesting static Tick nextTick() { Calendar calendar = new GregorianCalendar(); calendar.setTimeInMillis(EnvironmentEdgeManager.currentTime()); @@ -54,7 +52,6 @@ public class CurrentHourProvider { calendar.set(Calendar.MILLISECOND, 0); } - @VisibleForTesting static volatile Tick tick = nextTick(); public static int getCurrentHour() { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionPolicy.java index 1cc7dda0948..f60e97db483 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionPolicy.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionPolicy.java @@ -26,7 +26,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.OptionalLong; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HDFSBlocksDistribution; @@ -40,7 +39,7 @@ import org.apache.hadoop.hbase.util.ReflectionUtils; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + import org.apache.hbase.thirdparty.com.google.common.collect.Iterators; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hbase.thirdparty.com.google.common.collect.PeekingIterator; @@ -98,7 +97,7 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy { * Heuristics for guessing whether we need minor compaction. */ @Override - @VisibleForTesting + @InterfaceAudience.Private public boolean needsCompaction(Collection storeFiles, List filesCompacting) { ArrayList candidates = new ArrayList<>(storeFiles); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/throttle/StoreHotnessProtector.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/throttle/StoreHotnessProtector.java index ee6db3110bb..b907aa64009 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.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * StoreHotnessProtector is designed to help limit the concurrency of puts with dense columns, it * does best-effort to avoid exhausting all RS's handlers. When a lot of clients write requests with @@ -184,7 +182,6 @@ public class StoreHotnessProtector { return this.parallelPutToStoreThreadLimit > 0; } - @VisibleForTesting Map getPreparePutToStoreMap() { return preparePutToStoreMap; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java index ac99ea671e5..af5bfd5b03c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java @@ -48,7 +48,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantLock; - import org.apache.commons.lang3.mutable.MutableLong; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -87,13 +86,9 @@ import org.apache.htrace.core.TraceScope; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + import org.apache.hbase.thirdparty.com.google.common.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 @@ public abstract class AbstractFSWAL implements WAL { return newPath; } - @VisibleForTesting Path getOldPath() { long currentFilenum = this.filenum.get(); Path oldPath = null; @@ -785,7 +779,6 @@ public abstract class AbstractFSWAL implements WAL { return new Path(archiveDir, p.getName()); } - @VisibleForTesting protected void archiveLogFile(final Path p) throws IOException { Path newPath = getWALArchivePath(this.walArchiveDir, p); // Tell our listeners that a log is going to be archived. @@ -837,7 +830,6 @@ public abstract class AbstractFSWAL implements WAL { * @return the passed in newPath * @throws IOException if there is a problem flushing or closing the underlying FS */ - @VisibleForTesting Path replaceWriter(Path oldPath, Path newPath, W nextWriter) throws IOException { try (TraceScope scope = TraceUtil.createTrace("FSHFile.replaceWriter")) { doReplaceWriter(oldPath, newPath, nextWriter); @@ -941,7 +933,6 @@ public abstract class AbstractFSWAL implements WAL { * Get the backing files associated with this WAL. * @return may be null if there are no files. */ - @VisibleForTesting FileStatus[] getFiles() throws IOException { return CommonFSUtils.listStatus(fs, walDir, ourFiles); } @@ -1040,7 +1031,6 @@ public abstract class AbstractFSWAL implements WAL { /** * Exposed for testing only. Use to tricks like halt the ring buffer appending. */ - @VisibleForTesting protected void atHeadOfRingBufferEventHandlerAppend() { // Noop } @@ -1295,13 +1285,11 @@ public abstract class AbstractFSWAL implements WAL { /** * This method gets the pipeline for the current WAL. */ - @VisibleForTesting abstract DatanodeInfo[] getPipeline(); /** * This method gets the datanode replication count for the current WAL. */ - @VisibleForTesting abstract int getLogReplication(); private static void split(final Configuration conf, final Path p) throws IOException { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java index bf5b96dfce1..1279c2f31e8 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.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.io.Closeables; import org.apache.hbase.thirdparty.io.netty.channel.Channel; import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup; @@ -61,7 +60,6 @@ public class DualAsyncFSWAL extends AsyncFSWAL { } // 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 fe910aa067b..e2320dbf3c8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java @@ -40,7 +40,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; @@ -64,9 +63,8 @@ import org.apache.htrace.core.TraceScope; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; -import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; /** * The default implementation of FSWAL. @@ -204,13 +202,11 @@ public class FSHLog extends AbstractFSWAL { * @param logDir dir where wals are stored * @param conf configuration to use */ - @VisibleForTesting public FSHLog(final FileSystem fs, final Path root, final String logDir, final Configuration conf) throws IOException { this(fs, root, logDir, HConstants.HREGION_OLDLOGDIR_NAME, conf, null, true, null, null); } - @VisibleForTesting public FSHLog(final FileSystem fs, Abortable abortable, final Path root, final String logDir, final Configuration conf) throws IOException { this(fs, abortable, root, logDir, HConstants.HREGION_OLDLOGDIR_NAME, conf, null, true, null, @@ -283,7 +279,6 @@ public class FSHLog extends AbstractFSWAL { * removed. * @return null if underlying stream is not ready. */ - @VisibleForTesting OutputStream getOutputStream() { FSDataOutputStream fsdos = this.hdfs_out; return fsdos != null ? fsdos.getWrappedStream() : null; @@ -321,14 +316,12 @@ public class FSHLog extends AbstractFSWAL { * Used to manufacture race condition reliably. For testing only. * @see #beforeWaitOnSafePoint() */ - @VisibleForTesting protected void afterCreatingZigZagLatch() { } /** * @see #afterCreatingZigZagLatch() */ - @VisibleForTesting protected void beforeWaitOnSafePoint() { } @@ -777,7 +770,6 @@ public class FSHLog extends AbstractFSWAL { return logRollNeeded; } - @VisibleForTesting protected long getSequenceOnRingBuffer() { return this.disruptor.getRingBuffer().next(); } @@ -787,7 +779,6 @@ public class FSHLog extends AbstractFSWAL { return publishSyncOnRingBuffer(sequence, forceSync); } - @VisibleForTesting protected SyncFuture publishSyncOnRingBuffer(long sequence, boolean forceSync) { // here we use ring buffer sequence as transaction id SyncFuture syncFuture = getSyncFuture(sequence, forceSync); @@ -814,7 +805,6 @@ public class FSHLog extends AbstractFSWAL { * patch. */ @Override - @VisibleForTesting int getLogReplication() { try { // in standalone mode, it will return 0 @@ -855,7 +845,6 @@ public class FSHLog extends AbstractFSWAL { } } - @VisibleForTesting boolean isLowReplicationRollEnabled() { return lowReplicationRollEnabled; } @@ -1211,12 +1200,10 @@ public class FSHLog extends AbstractFSWAL { return new DatanodeInfo[0]; } - @VisibleForTesting Writer getWriter() { return this.writer; } - @VisibleForTesting void setWriter(Writer writer) { this.writer = writer; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java index 44c96dee761..ca51ec0c568 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java @@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.wal.WALKeyImpl; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils; /** @@ -91,7 +90,6 @@ class FSWALEntry extends Entry { } } - @VisibleForTesting static Set collectFamilies(List cells) { if (CollectionUtils.isEmpty(cells)) { return Collections.emptySet(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWAL.java index ee99aa30f98..f23dae21570 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 @@ public class MetricsWAL implements WALActionsListener { this(CompatibilitySingletonFactory.getInstance(MetricsWALSource.class)); } - @VisibleForTesting MetricsWAL(MetricsWALSource s) { this.source = s; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceIdAccounting.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceIdAccounting.java index 7146ca74392..5c6fcd2d1aa 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.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * Accounting of sequence ids per region and then by column family. So we can keep our accounting * current, call startCacheFlush and then finishedCacheFlush or abortCacheFlush so this instance can @@ -240,7 +238,6 @@ class SequenceIdAccounting { } } - @VisibleForTesting ConcurrentMap getOrCreateLowestSequenceIds(byte[] encodedRegionName) { // Intentionally, this access is done outside of this.regionSequenceIdLock. Done per append. return computeIfAbsent(this.lowestUnflushedSequenceIds, encodedRegionName, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ChainWALEmptyEntryFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ChainWALEmptyEntryFilter.java index 19fd0c77e7e..49236478071 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 java.util.List; 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 @@ public class ChainWALEmptyEntryFilter extends ChainWALEntryFilter { * * @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 b08c99098c5..86786856f21 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.List; 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.AsyncRegionServerAdmin; 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.apache.zookeeper.KeeperException.SessionExpiredException; 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 @@ public abstract class HBaseReplicationEndpoint extends BaseReplicationEndpoint badReportCounts.remove(sinkPeer.getServerName()); } - @VisibleForTesting List 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 123a036ca0f..81be5a3e3a0 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.Bytes; 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. *

@@ -59,7 +57,6 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti @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 @@ public final class ReplicationBarrierFamilyFormat { } } - @VisibleForTesting public static byte[] getParentsBytes(List parents) { ByteArrayOutputStream bos = new ByteArrayOutputStream(); Iterator 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 3271696e110..f06b29ccdef 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java @@ -34,7 +34,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.base.Predicate; import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; @@ -110,7 +109,7 @@ public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate { } } - @VisibleForTesting + @InterfaceAudience.Private public void setConf(Configuration conf, ZKWatcher zk) { super.setConf(conf); try { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java index 4e0be94a756..8f016bcb912 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 @@ package org.apache.hadoop.hbase.replication.master; 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.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.base.Predicate; import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; @@ -105,7 +103,7 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate { } } - @VisibleForTesting + @InterfaceAudience.Private public void setConf(Configuration conf, ZKWatcher zk) { super.setConf(conf); try { @@ -115,7 +113,8 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate { 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 56bccc09cdc..c77f74fe5c1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java @@ -39,7 +39,6 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import java.util.stream.Stream; - import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Abortable; @@ -66,7 +65,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; /** @@ -263,7 +261,6 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi /** * Check if there's an {@link TableNotFoundException} in the caused by stacktrace. */ - @VisibleForTesting public static boolean isTableNotFoundException(Throwable io) { if (io instanceof RemoteException) { io = ((RemoteException) io).unwrapRemoteException(); @@ -282,7 +279,6 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi /** * Check if there's an {@link NoSuchColumnFamilyException} in the caused by stacktrace. */ - @VisibleForTesting public static boolean isNoSuchColumnFamilyException(Throwable io) { if (io instanceof RemoteException) { io = ((RemoteException) io).unwrapRemoteException(); @@ -298,7 +294,6 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi return false; } - @VisibleForTesting List> filterNotExistTableEdits(final List> oldEntryList) { List> entryList = new ArrayList<>(); Map existMap = new HashMap<>(); @@ -342,7 +337,6 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi return entryList; } - @VisibleForTesting List> filterNotExistColumnFamilyEdits(final List> oldEntryList) { List> entryList = new ArrayList<>(); Map> existColumnFamilyMap = new HashMap<>(); @@ -556,7 +550,6 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi notifyStopped(); } - @VisibleForTesting protected int replicateEntries(List entries, int batchIndex, int timeout) throws IOException { SinkPeer sinkPeer = null; @@ -612,7 +605,6 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi return batchIndex; } - @VisibleForTesting protected Callable createReplicator(List entries, int batchIndex, int timeout) { return isSerial ? () -> serialReplicateRegionEntries(entries, batchIndex, timeout) : () -> replicateEntries(entries, batchIndex, timeout); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java index 0f73576feaf..098418da950 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.HBaseInterfaceAudience; import org.apache.hadoop.hbase.metrics.BaseSource; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * This class is for maintaining the various replication statistics for a source and publishing them * through the metrics interfaces. @@ -450,7 +448,7 @@ public class MetricsSource implements BaseSource { return globalSourceSource.getMetricsName(); } - @VisibleForTesting + @InterfaceAudience.Private public Map getSingleSourceSourceByTable() { return singleSourceSourceByTable; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java index 9e5fb945b04..423ec0e0005 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.hadoop.hbase.wal.WAL.Entry; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + import org.apache.hbase.thirdparty.com.google.common.collect.Lists; /** @@ -140,7 +140,6 @@ public class ReplicationSource implements ReplicationSourceInterface { private long defaultBandwidth; private long currentBandwidth; private WALFileLengthProvider walFileLengthProvider; - @VisibleForTesting protected final ConcurrentHashMap workerThreads = new ConcurrentHashMap<>(); @@ -455,7 +454,6 @@ public class ReplicationSource implements ReplicationSourceInterface { * Call after {@link #initializeWALEntryFilter(UUID)} else it will be null. * @return WAL Entry Filter Chain to use on WAL files filtering *out* WALEntry edits. */ - @VisibleForTesting WALEntryFilter getWalEntryFilter() { return walEntryFilter; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java index e323304beef..95157ca98d9 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.yetus.audience.InterfaceAudience; import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + import org.apache.hbase.thirdparty.com.google.common.collect.Sets; import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -197,7 +197,6 @@ public class ReplicationSourceManager implements ReplicationListener { * this server (in case it later gets moved back). We synchronize on this instance testing for * presence and if absent, while creating so only created and started once. */ - @VisibleForTesting AtomicReference catalogReplicationSource = new AtomicReference<>(); /** @@ -395,7 +394,6 @@ public class ReplicationSourceManager implements ReplicationListener { * @param peerId the id of the replication peer * @return the source that was created */ - @VisibleForTesting ReplicationSourceInterface addSource(String peerId) throws IOException { ReplicationPeer peer = replicationPeers.getPeer(peerId); ReplicationSourceInterface src = createSource(peerId, peer); @@ -673,7 +671,6 @@ public class ReplicationSourceManager implements ReplicationListener { * @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 class ReplicationSourceManager implements ReplicationListener { } // public because of we call it in TestReplicationEmptyWALRecovery - @VisibleForTesting public void preLogRoll(Path newLog) throws IOException { String logName = newLog.getName(); String logPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(logName); @@ -827,7 +823,6 @@ public class ReplicationSourceManager implements ReplicationListener { } // public because of we call it in TestReplicationEmptyWALRecovery - @VisibleForTesting public void postLogRoll(Path newLog) throws IOException { // This only updates the sources we own, not the recovered ones for (ReplicationSourceInterface source : this.sources.values()) { @@ -879,7 +874,6 @@ public class ReplicationSourceManager implements ReplicationListener { // the rs will abort (See HBASE-20475). private final Map peersSnapshot; - @VisibleForTesting public NodeFailoverWorker(ServerName deadRS) { super("Failover-for-" + deadRS); this.deadRS = deadRS; @@ -1034,7 +1028,6 @@ public class ReplicationSourceManager implements ReplicationListener { * Get a copy of the wals of the normal sources on this rs * @return a sorted set of wal names */ - @VisibleForTesting public Map>> getWALs() { return Collections.unmodifiableMap(walsById); } @@ -1043,7 +1036,6 @@ public class ReplicationSourceManager implements ReplicationListener { * Get a copy of the wals of the recovered sources on this rs * @return a sorted set of wal names */ - @VisibleForTesting Map>> getWalsByIdRecoveredQueues() { return Collections.unmodifiableMap(walsByIdRecoveredQueues); } @@ -1068,12 +1060,10 @@ public class ReplicationSourceManager implements ReplicationListener { * Get the normal source for a given peer * @return the normal source for the give peer if it exists, otherwise null. */ - @VisibleForTesting public ReplicationSourceInterface getSource(String peerId) { return this.sources.get(peerId); } - @VisibleForTesting List getAllQueues() throws IOException { List allQueues = Collections.emptyList(); try { @@ -1084,21 +1074,18 @@ public class ReplicationSourceManager implements ReplicationListener { return allQueues; } - @VisibleForTesting int getSizeOfLatestPath() { synchronized (latestPaths) { return latestPaths.size(); } } - @VisibleForTesting Set 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 d25ab072eb1..a38a96a3816 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALActionListener.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALActionListener.java @@ -27,8 +27,6 @@ import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.wal.WALKeyImpl; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * Used to receive new wals. */ @@ -65,7 +63,6 @@ class ReplicationSourceWALActionListener implements WALActionsListener { * @param logKey Key that may get scoped according to its edits * @param logEdit Edits used to lookup the scopes */ - @VisibleForTesting static void scopeWALEdits(WALKey logKey, WALEdit logEdit, Configuration conf) { // For bulk load replication we need meta family to know the file we want to replicate. if (ReplicationUtils.isReplicationForBulkLoadDataEnabled(conf)) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java index 27d407ebde2..89db7695fe2 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.List; 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.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap; import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @@ -416,7 +414,6 @@ public class RSGroupBasedLoadBalancer implements LoadBalancer { 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 16a44ad76cc..bae2ba5acf9 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.Collection; 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.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.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. 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 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager { 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 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager { * parent region cases. This method is invoked by {@link #balanceRSGroup} * @return A clone of current assignments for this group. */ - @VisibleForTesting Map>> getRSGroupAssignmentsByTable( TableStateManager tableStateManager, String groupName) throws IOException { Map>> result = Maps.newHashMap(); diff --git a/hbase-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 172561dea19..aec38ee4905 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.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLineParser; import org.apache.hbase.thirdparty.org.apache.commons.cli.DefaultParser; @@ -49,7 +48,7 @@ public class RSGroupMajorCompactionTTL extends MajorCompactorTTL { private static final Logger LOG = LoggerFactory.getLogger(RSGroupMajorCompactionTTL.class); - @VisibleForTesting + @InterfaceAudience.Private RSGroupMajorCompactionTTL() { super(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/PermissionStorage.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/PermissionStorage.java index 1006e54cd9d..268bc36fc45 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.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap; import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @@ -501,7 +500,6 @@ public final class PermissionStorage { null, false); } - @VisibleForTesting public static ListMultimap getNamespacePermissions(Configuration conf, String namespace) throws IOException { return getPermissions(conf, Bytes.toBytes(toNamespaceEntry(namespace)), null, null, null, null, @@ -780,7 +778,6 @@ public final class PermissionStorage { return list; } - @VisibleForTesting public static ListMultimap readUserPermission(byte[] data, Configuration conf) throws DeserializationException { if (ProtobufUtil.isPBMagicPrefix(data)) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SnapshotScannerHDFSAclCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SnapshotScannerHDFSAclCleaner.java index 8ce4266748c..72da07cee5e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SnapshotScannerHDFSAclCleaner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SnapshotScannerHDFSAclCleaner.java @@ -25,7 +25,6 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate; @@ -34,8 +33,6 @@ import org.apache.yetus.audience.InterfaceStability; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * Implementation of a file cleaner that checks if a empty directory with no subdirs and subfiles is * deletable when user scan snapshot feature is enabled @@ -98,7 +95,7 @@ public class SnapshotScannerHDFSAclCleaner extends BaseHFileCleanerDelegate { } } - @VisibleForTesting + @InterfaceAudience.Private static boolean isArchiveDataDir(Path path) { if (path != null && path.getName().equals(HConstants.BASE_NAMESPACE_DIR)) { Path parent = path.getParent(); @@ -107,12 +104,12 @@ public class SnapshotScannerHDFSAclCleaner extends BaseHFileCleanerDelegate { return false; } - @VisibleForTesting + @InterfaceAudience.Private static boolean isArchiveNamespaceDir(Path path) { return path != null && isArchiveDataDir(path.getParent()); } - @VisibleForTesting + @InterfaceAudience.Private static boolean isArchiveTableDir(Path path) { return path != null && isArchiveNamespaceDir(path.getParent()); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SnapshotScannerHDFSAclController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SnapshotScannerHDFSAclController.java index e52134e7d06..acb6940697a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SnapshotScannerHDFSAclController.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SnapshotScannerHDFSAclController.java @@ -26,7 +26,6 @@ import java.util.List; import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; @@ -65,7 +64,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.collect.Sets; /** @@ -542,7 +540,7 @@ public class SnapshotScannerHDFSAclController implements MasterCoprocessor, Mast return isSet; } - @VisibleForTesting + @InterfaceAudience.Private boolean checkInitialized(String operation) { if (initialized) { if (aclTableInitialized) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/ZKSecretWatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/ZKSecretWatcher.java index 50f87843903..2398ba4031e 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 @@ public class ZKSecretWatcher extends ZKListener { * get token keys parent node * @return token keys parent node */ - @VisibleForTesting String getKeysParentZNode() { return keysParentZNode; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java index 9df33e13132..5e82cad6b49 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java @@ -51,12 +51,11 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.CommonFSUtils; import org.apache.hadoop.hbase.util.FSTableDescriptors; import org.apache.hadoop.hbase.util.Threads; -import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.hbase.thirdparty.com.google.protobuf.CodedInputStream; import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException; @@ -197,7 +196,6 @@ public final class SnapshotManifest { addMobRegion(regionInfo, visitor); } - @VisibleForTesting protected void addMobRegion(RegionInfo regionInfo, RegionVisitor visitor) throws IOException { // 1. dump region meta info into the snapshot directory final String snapshotName = desc.getName(); @@ -245,7 +243,6 @@ public final class SnapshotManifest { addRegion(region, visitor); } - @VisibleForTesting protected void addRegion(final HRegion region, RegionVisitor visitor) throws IOException { // 1. dump region meta info into the snapshot directory final String snapshotName = desc.getName(); @@ -294,7 +291,6 @@ public final class SnapshotManifest { addRegion(tableDir, regionInfo, visitor); } - @VisibleForTesting protected void addRegion(final Path tableDir, final RegionInfo regionInfo, RegionVisitor visitor) throws IOException { boolean isMobRegion = MobUtils.isMobRegionInfo(regionInfo); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java index a39ecafdd02..ec9a59c7bf5 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.FutureUtils; 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 @@ public class BulkLoadHFilesTool extends Configured implements BulkLoadHFiles, To * @param lqis hfiles should be loaded * @return empty list if success, list of items to retry on recoverable failure */ - @VisibleForTesting + @InterfaceAudience.Private protected CompletableFuture> tryAtomicRegionLoad( final AsyncClusterConnection conn, final TableName tableName, boolean copyFiles, final byte[] first, Collection lqis) { @@ -434,7 +433,7 @@ public class BulkLoadHFilesTool extends Configured implements BulkLoadHFiles, To *

* protected for testing. */ - @VisibleForTesting + @InterfaceAudience.Private protected void bulkLoadPhase(AsyncClusterConnection conn, TableName tableName, Deque queue, Multimap regionGroups, boolean copyFiles, Map item2RegionMap) throws IOException { @@ -662,7 +661,7 @@ public class BulkLoadHFilesTool extends Configured implements BulkLoadHFiles, To * protected for testing * @throws IOException if an IO failure is encountered */ - @VisibleForTesting + @InterfaceAudience.Private protected Pair, String> groupOrSplit(AsyncClusterConnection conn, TableName tableName, Multimap regionGroups, LoadQueueItem item, List> startEndKeys) throws IOException { @@ -717,7 +716,7 @@ public class BulkLoadHFilesTool extends Configured implements BulkLoadHFiles, To * 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 b49d2cebede..89ad398c0fb 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java @@ -23,7 +23,6 @@ package org.apache.hadoop.hbase.tool; import java.util.Map; import java.util.concurrent.ExecutorService; import org.apache.hadoop.conf.Configuration; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.yetus.audience.InterfaceAudience; @InterfaceAudience.Public @@ -33,7 +32,7 @@ public interface Canary { return new CanaryTool(conf, executor); } - @VisibleForTesting + @InterfaceAudience.Private static Canary create(Configuration conf, ExecutorService executor, CanaryTool.Sink sink) { return new CanaryTool(conf, executor, sink); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/CanaryTool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/CanaryTool.java index 59e0e6cc791..f2c2d467ec0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/CanaryTool.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/CanaryTool.java @@ -99,7 +99,7 @@ import org.apache.zookeeper.client.ConnectStringParser; import org.apache.zookeeper.data.Stat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + import org.apache.hbase.thirdparty.com.google.common.collect.Lists; /** @@ -791,7 +791,7 @@ public class CanaryTool implements Tool, Canary { this(executor, null); } - @VisibleForTesting + @InterfaceAudience.Private CanaryTool(ExecutorService executor, Sink sink) { this.executor = executor; this.sink = sink; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/coprocessor/CoprocessorValidator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/coprocessor/CoprocessorValidator.java index 0e76f0ed8c2..766224e5d38 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/coprocessor/CoprocessorValidator.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/coprocessor/CoprocessorValidator.java @@ -37,7 +37,6 @@ import java.util.Optional; import java.util.regex.Pattern; import java.util.stream.Collectors; import java.util.stream.Stream; - import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.client.Admin; @@ -53,7 +52,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS) @@ -166,7 +164,7 @@ public class CoprocessorValidator extends AbstractHBaseTool { validateClasses(classLoader, Arrays.asList(classNames), violations); } - @VisibleForTesting + @InterfaceAudience.Private protected void validateTables(ClassLoader classLoader, Admin admin, Pattern pattern, List violations) throws IOException { List tableDescriptors = admin.listTableDescriptors(pattern); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterUtil.java index b932c4d9237..c7afb0e5f91 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterUtil.java @@ -29,8 +29,6 @@ import org.apache.hadoop.hbase.nio.ByteBuff; import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * Utility methods related to BloomFilters */ @@ -91,7 +89,6 @@ public final class BloomFilterUtil { * This gets used in {@link #contains(ByteBuff, int, int, Hash, int, HashKey)} * @param random The random number source to use, or null to compute actual hashes */ - @VisibleForTesting public static void setRandomGeneratorForTest(Random random) { randomGeneratorForTest = random; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java index 937069f92be..4fb231234e6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java @@ -50,7 +50,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.primitives.Ints; /** @@ -80,9 +79,7 @@ public class FSTableDescriptors implements TableDescriptors { private final boolean usecache; private volatile boolean fsvisited; - @VisibleForTesting long cachehits = 0; - @VisibleForTesting long invocations = 0; /** @@ -117,7 +114,6 @@ public class FSTableDescriptors implements TableDescriptors { this.usecache = usecache; } - @VisibleForTesting public static void tryUpdateMetaTableDescriptor(Configuration conf) throws IOException { tryUpdateAndGetMetaTableDescriptor(conf, CommonFSUtils.getCurrentFileSystem(conf), CommonFSUtils.getRootDir(conf)); @@ -192,7 +188,6 @@ public class FSTableDescriptors implements TableDescriptors { .setPriority(Coprocessor.PRIORITY_SYSTEM).build()); } - @VisibleForTesting protected boolean isUsecache() { return this.usecache; } @@ -299,7 +294,6 @@ public class FSTableDescriptors implements TableDescriptors { } } - @VisibleForTesting Path updateTableDescriptor(TableDescriptor td) throws IOException { TableName tableName = td.getTableName(); Path tableDir = getTableDir(tableName); @@ -410,7 +404,6 @@ public class FSTableDescriptors implements TableDescriptors { /** * Compare {@link FileStatus} instances by {@link Path#getName()}. Returns in reverse order. */ - @VisibleForTesting static final Comparator TABLEINFO_FILESTATUS_COMPARATOR = new Comparator() { @Override @@ -422,7 +415,6 @@ public class FSTableDescriptors implements TableDescriptors { /** * Return the table directory in HDFS */ - @VisibleForTesting Path getTableDir(final TableName tableName) { return CommonFSUtils.getTableDir(rootdir, tableName); } @@ -437,7 +429,6 @@ public class FSTableDescriptors implements TableDescriptors { /** * Width of the sequenceid that is a suffix on a tableinfo file. */ - @VisibleForTesting static final int WIDTH_OF_SEQUENCE_ID = 10; /** @@ -467,7 +458,6 @@ public class FSTableDescriptors implements TableDescriptors { * @param p Path to a .tableinfo file. * @return The current editid or 0 if none found. */ - @VisibleForTesting static int getTableInfoSequenceId(final Path p) { if (p == null) { return 0; @@ -487,7 +477,6 @@ public class FSTableDescriptors implements TableDescriptors { * @param sequenceid * @return Name of tableinfo file. */ - @VisibleForTesting static String getTableInfoFileName(final int sequenceid) { return TABLEINFO_FILE_PREFIX + "." + formatTableInfoSequenceId(sequenceid); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java index 31e7d3fb8b7..413b6ba7862 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.hdfs.protocol.HdfsConstants; 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 @@ public final class FSUtils { private static final int DEFAULT_THREAD_POOLSIZE = 2; /** Set to true on Windows platforms */ - @VisibleForTesting // currently only used in testing. TODO refactor into a test class + // currently only used in testing. TODO refactor into a test class public static final boolean WINDOWS = System.getProperty("os.name").startsWith("Windows"); private FSUtils() { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java index a52f3fb1c17..4ea4c9e14df 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.security.UserGroupInformation; import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; +import org.apache.hbase.thirdparty.com.google.common.base.Joiner; +import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; +import org.apache.hbase.thirdparty.com.google.common.collect.Sets; import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; @@ -141,12 +145,6 @@ import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; -import org.apache.hbase.thirdparty.com.google.common.base.Joiner; -import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; -import org.apache.hbase.thirdparty.com.google.common.collect.Lists; -import org.apache.hbase.thirdparty.com.google.common.collect.Sets; - /** * 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 @@ public class HBaseFsck extends Configured implements Closeable { * Here is where hbase-1.x used to default the lock for hbck1. * It puts in place a lock when it goes to write/make changes. */ - @VisibleForTesting + @InterfaceAudience.Private public static final String HBCK_LOCK_FILE = "hbase-hbck.lock"; private static final int DEFAULT_MAX_LOCK_FILE_ATTEMPTS = 5; private static final int DEFAULT_LOCK_FILE_ATTEMPT_SLEEP_INTERVAL = 200; // milliseconds @@ -400,7 +398,7 @@ public class HBaseFsck extends Configured implements Closeable { /** * @return Return the tmp dir this tool writes too. */ - @VisibleForTesting + @InterfaceAudience.Private public static Path getTmpDir(Configuration conf) throws IOException { return new Path(CommonFSUtils.getRootDir(conf), HConstants.HBASE_TEMP_DIRECTORY); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/IdReadWriteLock.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/IdReadWriteLock.java index 8b05fa9dbe3..10cc4e98d39 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 @@ package org.apache.hadoop.hbase.util; 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 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti public abstract class IdReadWriteLock { 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 e119784fb89..179b7d4a732 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.ConcurrentHashMap; 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 extends IdReadWriteLock { final ConcurrentHashMap 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 130ad145467..5492a8537d2 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 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 IdReadWriteLockWithObjectPool extends IdReadWriteLock{ // The number of lock we want to easily support. It's not a maximum. @@ -84,7 +82,6 @@ public class IdReadWriteLockWithObjectPool extends IdReadWriteLock{ } /** For testing */ - @VisibleForTesting int purgeAndGetEntryPoolSize() { gc(); Threads.sleep(200); @@ -97,7 +94,6 @@ public class IdReadWriteLockWithObjectPool extends IdReadWriteLock{ 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 9d7cb566c65..9ade12d578c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/LossyCounting.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/LossyCounting.java @@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; /** @@ -115,7 +115,6 @@ public class LossyCounting { /** * sweep low frequency data */ - @VisibleForTesting public void sweep() { for(Map.Entry entry : data.entrySet()) { if(entry.getValue() < currentTerm) { @@ -168,7 +167,7 @@ public class LossyCounting { } } - @VisibleForTesting public Future getSweepFuture() { + public Future getSweepFuture() { return fut.get(); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java index 03e674d54cc..2f7d351ff99 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.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils; @@ -141,7 +140,7 @@ public class RegionMover extends AbstractHBaseTool implements Closeable { private String excludeFile = null; private String designatedFile = null; private String defaultDir = System.getProperty("java.io.tmpdir"); - @VisibleForTesting + @InterfaceAudience.Private final int port; private final Configuration conf; @@ -385,8 +384,8 @@ public class RegionMover extends AbstractHBaseTool implements Closeable { return waitTaskToFinish(unloadPool, unloadTask, "unloading"); } - @VisibleForTesting - Collection filterRSGroupServers(RSGroupInfo rsgroup, + @InterfaceAudience.Private + Collection filterRSGroupServers(RSGroupInfo rsgroup, Collection 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 c0d34d9397e..9be182d245f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/ClusterCompactionQueues.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/ClusterCompactionQueues.java @@ -25,7 +25,7 @@ import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.hadoop.hbase.ServerName; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hbase.thirdparty.com.google.common.collect.Maps; import org.apache.hbase.thirdparty.com.google.common.collect.Sets; @@ -82,7 +82,7 @@ class ClusterCompactionQueues { } } - @VisibleForTesting List getQueue(ServerName serverName) { + List getQueue(ServerName serverName) { lock.readLock().lock(); try { return compactionQueues.get(serverName); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionRequest.java index aecfc37d8ef..22ec6cb89ec 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionRequest.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionRequest.java @@ -38,7 +38,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.collect.Sets; @InterfaceAudience.Private @@ -55,7 +54,6 @@ class MajorCompactionRequest { this.region = region; } - @VisibleForTesting MajorCompactionRequest(Configuration configuration, RegionInfo region, Set stores) { this(configuration, region); @@ -81,7 +79,6 @@ class MajorCompactionRequest { this.stores = stores; } - @VisibleForTesting Optional createRequest(Configuration configuration, Set stores, long timestamp) throws IOException { Set familiesToCompact = getStoresRequiringCompaction(stores, timestamp); @@ -145,7 +142,6 @@ class MajorCompactionRequest { return false; } - @VisibleForTesting Connection getConnection(Configuration configuration) throws IOException { return ConnectionFactory.createConnection(configuration); } @@ -166,13 +162,11 @@ class MajorCompactionRequest { } - @VisibleForTesting List 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 4d2b341ecd5..0eda4591841 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.io.IOException; 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.hadoop.hbase.regionserver.StoreFileInfo; 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 @@ public class MajorCompactionTTLRequest extends MajorCompactionRequest { return request.createRequest(conf, htd); } - @VisibleForTesting private Optional createRequest(Configuration conf, TableDescriptor htd) throws IOException { Map 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 151b49286c1..370a3e8de44 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.ExecutorService; 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.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.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 @@ public class MajorCompactor extends Configured implements Tool { 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 321cbe03386..7c92c530fe1 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.Arrays; 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.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.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 @@ public class MajorCompactorTTL extends MajorCompactor { 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 109e1107669..720e2c26aa4 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.ReadWriteLock; 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.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + import org.apache.hbase.thirdparty.com.google.common.collect.Lists; /** @@ -217,7 +216,6 @@ public abstract class AbstractFSWALProvider> implemen /** * returns the number of rolled WAL files. */ - @VisibleForTesting public static int getNumRolledLogFiles(WAL wal) { return ((AbstractFSWAL) wal).getNumRolledLogFiles(); } @@ -225,7 +223,6 @@ public abstract class AbstractFSWALProvider> implemen /** * returns the size of rolled WAL files. */ - @VisibleForTesting public static long getLogFileSize(WAL wal) { return ((AbstractFSWAL) wal).getLogFileSize(); } @@ -233,7 +230,6 @@ public abstract class AbstractFSWALProvider> implemen /** * return the current filename from the current wal. */ - @VisibleForTesting public static Path getCurrentFileName(final WAL wal) { return ((AbstractFSWAL) wal).getCurrentFileName(); } @@ -241,7 +237,6 @@ public abstract class AbstractFSWALProvider> implemen /** * request a log roll, but don't actually do it. */ - @VisibleForTesting static void requestLogRoll(final WAL wal) { ((AbstractFSWAL) wal).requestLogRoll(); } @@ -249,7 +244,6 @@ public abstract class AbstractFSWALProvider> implemen // 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 @@ public abstract class AbstractFSWALProvider> implemen * @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 6348e5cc882..0ca1219bd26 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.apache.yetus.audience.InterfaceAudience; 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 @@ class EntryBuffers { } } - @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 001e1a8ea95..f57ec31c531 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.Matcher; 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.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.Streams; import org.apache.hbase.thirdparty.io.netty.channel.Channel; @@ -73,7 +71,6 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen 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 @@ public class SyncReplicationWALProvider implements WALProvider, PeerActionListen } } - @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 747b2770d45..2a434a73b67 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.hadoop.hbase.replication.regionserver.WALFileLengthProvider; 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 @@ public interface WAL extends Closeable, WALFileLengthProvider { * @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 1fae7a5d0ed..61f36fab74a 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.ClassSize; 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 @@ public class WALEdit implements HeapSize { * {@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 @@ public class WALEdit implements HeapSize { 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 @@ public class WALEdit implements HeapSize { 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 @@ public class WALEdit implements HeapSize { * @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 6964bb885b9..82e974deb35 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.hadoop.hbase.wal.WALProvider.Writer; 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 @@ public class WALFactory { this.abortable = null; } - @VisibleForTesting Providers getDefaultProvider() { return Providers.defaultProvider; } - @VisibleForTesting public Class getProviderClass(String key, String defaultValue) { try { Providers provider = Providers.valueOf(conf.get(key, defaultValue)); @@ -403,7 +400,6 @@ public class WALFactory { * 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 @@ public class WALFactory { * 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 33e034342d7..4c3fc4edc78 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.Map; 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.Bytes; 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 @@ public class WALKeyImpl implements WALKey { 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 clusterIds = new ArrayList<>(1); @@ -139,7 +137,7 @@ public class WALKeyImpl implements WALKey { 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 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 d392366ff5d..6361ffc4cb3 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.hadoop.hbase.zookeeper.ZKSplitLog; 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 @@ public final class WALSplitUtil { * @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 @@ public final class WALSplitUtil { return new Path(srcPath.getParent(), fileName); } - @VisibleForTesting static String formatRecoveredEditsFileName(final long seqid) { return String.format("%019d", seqid); } @@ -345,7 +344,6 @@ public final class WALSplitUtil { /** * 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 0bea35eaf89..6616a116b8b 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.Map; 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.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.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 @@ public class WALSplitter { 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 @@ public class WALSplitter { * which uses this method to do log splitting. * @return List of output files created by the split. */ - @VisibleForTesting public static List 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 @@ public class WALSplitter { * 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 a7bb8e6e648..f610ad6f07a 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.io.hfile.bucket.BucketCache; 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 @@ public class CacheTestUtils { 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 27a74c3ff97..9ba4e543840 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.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.util.concurrent.ThreadFactoryBuilder; @@ -331,7 +330,7 @@ public class ThriftServer extends Configured implements Tool { * 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 @@ public class ThriftServer extends Configured implements Tool { * 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 9259ec6101f..552ceecde74 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.ZooKeeperServer; 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 @@ public class MiniZooKeeperCluster { * * @return clientPortList the client port list */ - @VisibleForTesting public List getClientPortList() { + @InterfaceAudience.Private + public List getClientPortList() { return clientPortList; }