HBASE-24640 Purge use of VisibleForTesting (#2695)

Signed-off-by: Reid Chan <reidchan@apache.org>
Signed-off-by: Nick Dimiduk <ndimiduk@apache.org>
This commit is contained in:
Andrew Purtell 2020-11-25 14:10:12 -08:00 committed by GitHub
parent d81b541493
commit 1726160839
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
249 changed files with 243 additions and 1149 deletions

View File

@ -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;
}

View File

@ -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;

View File

@ -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) {

View File

@ -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 <code>hbase:meta</code> region as well as assignment information stored
* to <code>hbase:meta</code>.
@ -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<RegionInfo> getAllRegions(Connection connection,
boolean excludeOfflinedSplitParents) throws IOException {
List<Pair<RegionInfo, ServerName>> 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);

View File

@ -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 &gt; 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);

View File

@ -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();

View File

@ -59,7 +59,6 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.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(),

View File

@ -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;

View File

@ -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) {

View File

@ -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();

View File

@ -29,8 +29,6 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* The {@link ResultScanner} implementation for {@link AsyncTable}. It will fetch data automatically
* in background and cache it in memory. Typically the {@link #maxCacheSize} will be
@ -177,7 +175,6 @@ class AsyncTableResultScanner implements ResultScanner, AdvancedScanResultConsum
}
// used in tests to test whether the scanner has been suspended
@VisibleForTesting
synchronized boolean isSuspended() {
return resumer != null;
}

View File

@ -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;

View File

@ -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 {
* <p/>
* Will be called in {@code HBaseTestingUtility}.
*/
@VisibleForTesting
public static String getMasterAddr(Configuration conf) throws UnknownHostException {
String masterAddrFromConf = conf.get(MASTER_ADDRS_KEY);
if (!Strings.isNullOrEmpty(masterAddrFromConf)) {
@ -331,7 +329,6 @@ public class MasterRegistry implements ConnectionRegistry {
"getMasters()").thenApply(MasterRegistry::transformServerNames);
}
@VisibleForTesting
Set<ServerName> getParsedMasterServers() {
return masterAddr2Stub.keySet();
}

View File

@ -26,7 +26,6 @@ import com.codahale.metrics.JmxReporter;
import com.codahale.metrics.MetricRegistry;
import com.codahale.metrics.RatioGauge;
import com.codahale.metrics.Timer;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
@ -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<ServerName, ConcurrentMap<byte[], RegionStats>> 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<String, Timer> rpcTimers =
protected final ConcurrentMap<String, Timer> rpcTimers =
new ConcurrentHashMap<>(CAPACITY, LOAD_FACTOR, CONCURRENCY_LEVEL);
@VisibleForTesting protected final ConcurrentMap<String, Histogram> rpcHistograms =
protected final ConcurrentMap<String, Histogram> rpcHistograms =
new ConcurrentHashMap<>(CAPACITY * 2 /* tracking both request and response sizes */,
LOAD_FACTOR, CONCURRENCY_LEVEL);
private final ConcurrentMap<String, Counter> cacheDroppingExceptions =
new ConcurrentHashMap<>(CAPACITY, LOAD_FACTOR, CONCURRENCY_LEVEL);
@VisibleForTesting protected final ConcurrentMap<String, Counter> rpcCounters =
protected final ConcurrentMap<String, Counter> rpcCounters =
new ConcurrentHashMap<>(CAPACITY, LOAD_FACTOR, CONCURRENCY_LEVEL);
MetricsConnection(String scope, Supplier<ThreadPoolExecutor> batchPool,
@ -358,17 +354,14 @@ public class MetricsConnection implements StatisticTrackable {
this.reporter.start();
}
@VisibleForTesting
final String getExecutorPoolName() {
return name(getClass(), "executorPoolActiveThreads", scope);
}
@VisibleForTesting
final String getMetaPoolName() {
return name(getClass(), "metaPoolActiveThreads", scope);
}
@VisibleForTesting
MetricRegistry getMetricRegistry() {
return registry;
}

View File

@ -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<HRegionLocation> getRegionLocation(byte[] regionNameOrEncodedRegionName) {
if (regionNameOrEncodedRegionName == null) {
return failedFuture(new IllegalArgumentException("Passed region name can't be null"));

View File

@ -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<byte[], AtomicInteger> taskCounterPerRegion
= new ConcurrentSkipListMap<>(Bytes.BYTES_COMPARATOR);
@VisibleForTesting
final ConcurrentMap<ServerName, AtomicInteger> taskCounterPerServer = new ConcurrentHashMap<>();
/**
* The number of tasks simultaneously executed on the cluster.
@ -112,13 +105,11 @@ 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<RowChecker> 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);

View File

@ -24,6 +24,7 @@ import static org.apache.hadoop.hbase.client.RegionReplicaUtil.getRegionInfoForR
import static org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.lengthOfPBMagic;
import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
import static org.apache.hadoop.hbase.zookeeper.ZKMetadata.removeMetaData;
import java.io.IOException;
import java.util.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;
}

View File

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

View File

@ -37,8 +37,6 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.UnsafeAvailChecker;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* This is optimized version of a standard FuzzyRowFilter Filters data based on fuzzy row key.
* Performs fast-forwards during scanning. It takes pairs (row key, fuzzy info) to match row keys.
@ -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

View File

@ -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<T extends RpcConnection> implements RpcC
}
}
@VisibleForTesting
public static String getDefaultCodec(final Configuration c) {
// If "hbase.client.default.rpc.codec" is empty string -- you can't set it to null because
// Configuration will complain -- then no default codec (and we'll pb everything). Else
@ -248,7 +246,6 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
}
// for writing tests that want to throw exception when connecting.
@VisibleForTesting
boolean isTcpNoDelay() {
return tcpNoDelay;
}
@ -565,7 +562,6 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
/**
* Blocking rpc channel that goes via hbase rpc.
*/
@VisibleForTesting
public static class BlockingRpcChannelImplementation extends AbstractRpcChannel
implements BlockingRpcChannel {

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -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;
}

View File

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

View File

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

View File

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

View File

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

View File

@ -40,8 +40,6 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Utility methods helpful slinging {@link Cell} instances. It has more powerful and
* rich set of APIs than those in {@link CellUtil} for internal usage.
@ -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

View File

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

View File

@ -25,7 +25,6 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Maintains the set of all the classes which would like to get notified
@ -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);

View File

@ -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();

View File

@ -23,9 +23,7 @@ import java.io.OutputStream;
import java.security.GeneralSecurityException;
import java.security.Key;
import java.security.SecureRandom;
import javax.crypto.spec.SecretKeySpec;
import org.apache.hadoop.hbase.io.crypto.Cipher;
import org.apache.hadoop.hbase.io.crypto.CipherProvider;
import org.apache.hadoop.hbase.io.crypto.Context;
@ -36,7 +34,6 @@ import org.apache.yetus.audience.InterfaceStability;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
/**
@ -149,7 +146,6 @@ public class AES extends Cipher {
return d.createDecryptionStream(in);
}
@VisibleForTesting
SecureRandom getRNG() {
return rng;
}

View File

@ -24,9 +24,7 @@ import java.security.GeneralSecurityException;
import java.security.Key;
import java.security.SecureRandom;
import java.util.Properties;
import javax.crypto.spec.SecretKeySpec;
import org.apache.commons.crypto.cipher.CryptoCipherFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.io.crypto.Cipher;
@ -39,7 +37,6 @@ import org.apache.yetus.audience.InterfaceStability;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
@InterfaceAudience.Private
@ -159,7 +156,6 @@ public class CommonsCryptoAES extends Cipher {
return decryptor.createDecryptionStream(in);
}
@VisibleForTesting
SecureRandom getRNG() {
return rng;
}

View File

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

View File

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

View File

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

View File

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

View File

@ -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();

View File

@ -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<Bytes> {
// 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<Bytes> {
}
@VisibleForTesting
static Comparer<byte[]> lexicographicalComparerJavaImpl() {
return LexicographicalComparerHolder.PureJavaComparer.INSTANCE;
}
@ -1420,7 +1415,6 @@ public class Bytes implements Comparable<Bytes> {
* <p>Uses reflection to gracefully fall back to the Java implementation if
* {@code Unsafe} isn't available.
*/
@VisibleForTesting
static class LexicographicalComparerHolder {
static final String UNSAFE_COMPARER_NAME =
LexicographicalComparerHolder.class.getName() + "$UnsafeComparer";
@ -1470,7 +1464,6 @@ public class Bytes implements Comparable<Bytes> {
}
}
@VisibleForTesting
enum UnsafeComparer implements Comparer<byte[]> {
INSTANCE;

View File

@ -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;
}

View File

@ -28,7 +28,6 @@ import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FSDataOutputStreamBuilder;
@ -41,12 +40,10 @@ import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
/**
@ -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());
}

View File

@ -21,12 +21,10 @@ import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
/**
@ -213,12 +211,10 @@ public class IdLock {
}
}
@VisibleForTesting
void assertMapEmpty() {
assert map.isEmpty();
}
@VisibleForTesting
public void waitForWaiters(long id, int numWaiters) throws InterruptedException {
for (Entry entry;;) {
entry = map.get(id);

View File

@ -28,8 +28,6 @@ import java.nio.charset.Charset;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Utility class that handles ordered byte arrays. That is, unlike
* {@link Bytes}, these methods produce byte arrays which maintain the sort
@ -368,7 +366,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

View File

@ -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;

View File

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

View File

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

View File

@ -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<byte[], Response> run(final Configuration conf, final String[] args) throws Throwable {
String[] otherArgs = new GenericOptionsParser(conf, args).getRemainingArgs();
if (!ExportUtils.isValidArguements(args)) {

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -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;
}

View File

@ -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.

View File

@ -90,8 +90,6 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Writes HFiles. Passed Cells must arrive in order.
* Writes current time as the sequence id for the file. Sets the major compacted
@ -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<byte[], Algorithm> createFamilyCompressionMap(Configuration
conf) {
Map<byte[], String> 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<byte[], BloomType> createFamilyBloomTypeMap(Configuration conf) {
Map<byte[], String> 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<byte[], String> createFamilyBloomParamMap(Configuration conf) {
return createFamilyConfValueMap(conf, BLOOM_PARAM_FAMILIES_CONF_KEY);
}
/**
* Runs inside the task to deserialize column family to block size
* map from the configuration.
@ -717,7 +714,7 @@ public class HFileOutputFormat2
* @param conf to read the serialized values from
* @return a map from column family to the configured block size
*/
@VisibleForTesting
@InterfaceAudience.Private
static Map<byte[], Integer> createFamilyBlockSizeMap(Configuration conf) {
Map<byte[], String> stringMap = createFamilyConfValueMap(conf,
BLOCK_SIZE_FAMILIES_CONF_KEY);
@ -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<byte[], DataBlockEncoding> createFamilyDataBlockEncodingMap(
Configuration conf) {
Map<byte[], String> 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<ColumnFamilyDescriptor, String> fn,
List<TableDescriptor> 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<ColumnFamilyDescriptor, String> 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<ColumnFamilyDescriptor, String> 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<ColumnFamilyDescriptor, String> 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<ColumnFamilyDescriptor, String> 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<ColumnFamilyDescriptor, String> dataBlockEncodingDetails = familyDescriptor -> {
DataBlockEncoding encoding = familyDescriptor.getDataBlockEncoding();
if (encoding == null) {

View File

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

View File

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

View File

@ -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;

View File

@ -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<ImmutableBytesWritable
}
}
@VisibleForTesting
@InterfaceAudience.Private
static class TableSnapshotRegionRecordReader extends
RecordReader<ImmutableBytesWritable, Result> {
private TableSnapshotInputFormatImpl.RecordReader delegate =

View File

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

View File

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

View File

@ -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 {

View File

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

View File

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

View File

@ -56,7 +56,6 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
@ -639,7 +638,6 @@ public class ProcedureExecutor<TEnvironment> {
workerMonitorExecutor.sendStopSignal();
}
@VisibleForTesting
public void join() {
assert !isRunning() : "expected not running";
@ -1332,12 +1330,10 @@ public class ProcedureExecutor<TEnvironment> {
return procId;
}
@VisibleForTesting
protected long getLastProcId() {
return lastProcId.get();
}
@VisibleForTesting
public Set<Long> getActiveProcIds() {
return procedures.keySet();
}
@ -1932,17 +1928,14 @@ public class ProcedureExecutor<TEnvironment> {
return rollbackStack.get(rootProcId);
}
@VisibleForTesting
ProcedureScheduler getProcedureScheduler() {
return scheduler;
}
@VisibleForTesting
int getCompletedSize() {
return completed.size();
}
@VisibleForTesting
public IdLock getProcExecutionLock() {
return procExecutionLock;
}

View File

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

View File

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

View File

@ -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<TEnvironment, TState>
*/
private int previousState;
@VisibleForTesting
public enum Flow {
HAS_MORE_STATE,
NO_MORE_STATE,
@ -282,7 +278,6 @@ public abstract class StateMachineProcedure<TEnvironment, TState>
* sequentially. Some procedures may skip steps/ states, some may add intermediate steps in
* future.
*/
@VisibleForTesting
public int getCurrentStateId() {
return getStateId(getCurrentState());
}

View File

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

View File

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

View File

@ -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);
}

View File

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

View File

@ -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<Long, Long> getExecutorStatus(String poolName) {
MutableGaugeLong running = metricsRegistry.getGauge(poolName + "_running", 0L);
MutableGaugeLong queued = metricsRegistry.getGauge(poolName + "_queued", 0L);

View File

@ -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;
}

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -32,13 +32,11 @@ import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.hbase.monitoring.ThreadMonitoring;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ListenableFuture;
@ -85,7 +83,6 @@ public class ExecutorService {
* started with the same name, this throws a RuntimeException.
* @param name Name of the service to start.
*/
@VisibleForTesting
public void startExecutorService(String name, int maxThreads) {
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();
}

View File

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

View File

@ -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;

View File

@ -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;
}

View File

@ -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;
}

View File

@ -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)

View File

@ -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)

View File

@ -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();
/**

View File

@ -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);

View File

@ -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);
}

View File

@ -54,8 +54,6 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* 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;
}

View File

@ -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<String> getCachedFileNamesForTest() {
SortedSet<String> fileNames = new TreeSet<>();
for (BlockCacheKey cacheKey : map.keySet()) {
@ -1193,7 +1188,6 @@ public class LruBlockCache implements FirstLevelBlockCache {
return fileNames;
}
@VisibleForTesting
public Map<DataBlockEncoding, Integer> getEncodingCountsForTest() {
Map<DataBlockEncoding, Integer> counts = new EnumMap<>(DataBlockEncoding.class);
for (LruCachedBlock block : map.values()) {
@ -1204,7 +1198,6 @@ public class LruBlockCache implements FirstLevelBlockCache {
return counts;
}
@VisibleForTesting
Map<BlockCacheKey, LruCachedBlock> getMapForTests() {
return map;
}

View File

@ -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)

View File

@ -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<BlockCacheKey, Cacheable> cache;
/**

View File

@ -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<BlockCacheKey, BucketEntry> 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<BlockingQueue<RAMQueueEntry>> writerQueues = new ArrayList<>();
@VisibleForTesting
transient final WriterThread[] writerThreads;
/** Volatile boolean to track if free space is in process or not */
@ -185,7 +178,6 @@ 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.
* <p>
*/
@VisibleForTesting
transient final IdReadWriteLock<Long> offsetLock;
private final NavigableSet<BlockCacheKey> 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<RAMQueueEntry> 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<RAMQueueEntry> 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<RAMQueueEntry> getRAMQueueEntries(BlockingQueue<RAMQueueEntry> q,
List<RAMQueueEntry> receptacle) throws InterruptedException {
// Clear sets all entries to null and sets size to 0. We retain allocations. Presume it
@ -1349,7 +1332,6 @@ 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) {

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