HBASE-24640 [branch-1] Purge use of VisibleForTesting (#2697)
Signed-off-by: Reid Chan <reidchan@apache.org> Signed-off-by: Nick Dimiduk <ndimiduk@apache.org>
This commit is contained in:
parent
4640b2224d
commit
30d988f6e2
|
@ -17,7 +17,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
|
@ -668,7 +667,6 @@ public class MetaTableAccessor {
|
|||
* @param replicaId the replicaId of the region
|
||||
* @return a byte[] for server column qualifier
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public static byte[] getServerColumn(int replicaId) {
|
||||
return replicaId == 0
|
||||
? HConstants.SERVER_QUALIFIER
|
||||
|
@ -681,7 +679,6 @@ public class MetaTableAccessor {
|
|||
* @param replicaId the replicaId of the region
|
||||
* @return a byte[] for server start code column qualifier
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public static byte[] getStartCodeColumn(int replicaId) {
|
||||
return replicaId == 0
|
||||
? HConstants.STARTCODE_QUALIFIER
|
||||
|
@ -694,7 +691,6 @@ public class MetaTableAccessor {
|
|||
* @param replicaId the replicaId of the region
|
||||
* @return a byte[] for seqNum column qualifier
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public static byte[] getSeqNumColumn(int replicaId) {
|
||||
return replicaId == 0
|
||||
? HConstants.SEQNUM_QUALIFIER
|
||||
|
@ -708,7 +704,6 @@ public class MetaTableAccessor {
|
|||
* @param serverColumn the column qualifier
|
||||
* @return an int for the replicaId
|
||||
*/
|
||||
@VisibleForTesting
|
||||
static int parseReplicaIdFromServerColumn(byte[] serverColumn) {
|
||||
String serverStr = Bytes.toString(serverColumn);
|
||||
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -30,7 +29,6 @@ import java.util.Date;
|
|||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
@ -436,7 +434,6 @@ class AsyncProcess {
|
|||
* @return pool if non null, otherwise returns this.pool if non null, otherwise throws
|
||||
* RuntimeException
|
||||
*/
|
||||
@VisibleForTesting
|
||||
ExecutorService getPool(ExecutorService pool) {
|
||||
if (pool != null) {
|
||||
return pool;
|
||||
|
@ -803,13 +800,11 @@ class AsyncProcess {
|
|||
* Runnable (that can be submitted to thread pool) that submits MultiAction to a
|
||||
* single server. The server call is synchronous, therefore we do it on a thread pool.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
class SingleServerRequestRunnable implements Runnable {
|
||||
private final MultiAction<Row> multiAction;
|
||||
private final int numAttempt;
|
||||
private final ServerName server;
|
||||
private final Set<PayloadCarryingServerCallable> callsInProgress;
|
||||
@VisibleForTesting
|
||||
SingleServerRequestRunnable(
|
||||
MultiAction<Row> multiAction, int numAttempt, ServerName server,
|
||||
Set<PayloadCarryingServerCallable> callsInProgress) {
|
||||
|
@ -988,7 +983,6 @@ class AsyncProcess {
|
|||
return callsInProgress;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
SingleServerRequestRunnable createSingleServerRequest(MultiAction<Row> multiAction, int numAttempt, ServerName server,
|
||||
Set<PayloadCarryingServerCallable> callsInProgress) {
|
||||
return new SingleServerRequestRunnable(multiAction, numAttempt, server, callsInProgress);
|
||||
|
@ -1387,7 +1381,6 @@ class AsyncProcess {
|
|||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
long getActionsInProgress() {
|
||||
return actionsInProgress.get();
|
||||
}
|
||||
|
@ -1798,7 +1791,6 @@ class AsyncProcess {
|
|||
/**
|
||||
* Create a callable. Isolated to be easily overridden in the tests.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
protected MultiServerCallable<Row> createCallable(final ServerName server,
|
||||
TableName tableName, final MultiAction<Row> multi) {
|
||||
return new MultiServerCallable<Row>(connection, tableName, server,
|
||||
|
@ -1806,7 +1798,6 @@ class AsyncProcess {
|
|||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
protected void updateStats(ServerName server, Map<byte[], MultiResponse.RegionResult> results) {
|
||||
boolean metrics = AsyncProcess.this.connection.getConnectionMetrics() != null;
|
||||
boolean stats = AsyncProcess.this.connection.getStatisticsTracker() != null;
|
||||
|
@ -1823,7 +1814,6 @@ class AsyncProcess {
|
|||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
<CResult> AsyncRequestFutureImpl<CResult> createAsyncRequestFuture(
|
||||
TableName tableName, List<Action<Row>> actions, long nonceGroup, ExecutorService pool,
|
||||
Batch.Callback<CResult> callback, Object[] results, boolean needResults,
|
||||
|
@ -1836,13 +1826,11 @@ class AsyncProcess {
|
|||
/**
|
||||
* Create a caller. Isolated to be easily overridden in the tests.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
protected RpcRetryingCaller<MultiResponse> createCaller(PayloadCarryingServerCallable callable,
|
||||
int rpcTimeout) {
|
||||
return rpcCallerFactory.<MultiResponse> newCaller(rpcTimeout);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
/** Waits until all outstanding tasks are done. Used in tests. */
|
||||
void waitUntilDone() throws InterruptedIOException {
|
||||
waitForMaximumCurrentTasks(0, null);
|
||||
|
@ -1855,7 +1843,6 @@ class AsyncProcess {
|
|||
}
|
||||
|
||||
// Break out this method so testable
|
||||
@VisibleForTesting
|
||||
void waitForMaximumCurrentTasks(int max, final AtomicLong tasksInProgress, final long id,
|
||||
String tableName) throws InterruptedIOException {
|
||||
long lastLog = EnvironmentEdgeManager.currentTime();
|
||||
|
@ -2011,7 +1998,6 @@ class AsyncProcess {
|
|||
/**
|
||||
* Collect all advices from checkers and make the final decision.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
static class RowCheckerHost {
|
||||
private final List<RowChecker> checkers;
|
||||
private boolean isEnd = false;
|
||||
|
@ -2064,7 +2050,6 @@ class AsyncProcess {
|
|||
/**
|
||||
* Provide a way to control the flow of rows iteration.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
interface RowChecker {
|
||||
enum ReturnCode {
|
||||
/**
|
||||
|
@ -2098,7 +2083,6 @@ class AsyncProcess {
|
|||
* Reduce the limit of heapsize for submitting quickly
|
||||
* if there is no running task.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
static class SubmittedSizeChecker implements RowChecker {
|
||||
private final long maxHeapSizeSubmit;
|
||||
private long heapSize = 0;
|
||||
|
@ -2128,7 +2112,6 @@ class AsyncProcess {
|
|||
/**
|
||||
* limit the max number of tasks in an AsyncProcess.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
static class TaskCountChecker implements RowChecker {
|
||||
private static final long MAX_WAITING_TIME = 1000; //ms
|
||||
private final Set<HRegionInfo> regionsIncluded = new HashSet<>();
|
||||
|
@ -2233,7 +2216,6 @@ class AsyncProcess {
|
|||
/**
|
||||
* limit the request size for each regionserver.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
static class RequestSizeChecker implements RowChecker {
|
||||
private final long maxHeapSizePerRequest;
|
||||
private final Map<ServerName, Long> serverRequestSizes = new HashMap<>();
|
||||
|
|
|
@ -16,7 +16,6 @@
|
|||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import static org.apache.hadoop.hbase.client.BufferedMutatorParams.UNSET;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
|
@ -67,16 +66,13 @@ public class BufferedMutatorImpl implements BufferedMutator {
|
|||
protected ClusterConnection connection; // non-final so can be overridden in test
|
||||
private final TableName tableName;
|
||||
private volatile Configuration conf;
|
||||
@VisibleForTesting
|
||||
final ConcurrentLinkedQueue<Mutation> writeAsyncBuffer = new ConcurrentLinkedQueue<Mutation>();
|
||||
@VisibleForTesting
|
||||
AtomicLong currentWriteBufferSize = new AtomicLong(0);
|
||||
|
||||
/**
|
||||
* Count the size of {@link BufferedMutatorImpl#writeAsyncBuffer}.
|
||||
* The {@link ConcurrentLinkedQueue#size()} is NOT a constant-time operation.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
final AtomicInteger undealtMutationCount = new AtomicInteger(0);
|
||||
private long writeBufferSize;
|
||||
/**
|
||||
|
@ -96,7 +92,6 @@ public class BufferedMutatorImpl implements BufferedMutator {
|
|||
private int writeRpcTimeout; // needed to pass in through AsyncProcess constructor
|
||||
private int operationTimeout;
|
||||
|
||||
@VisibleForTesting
|
||||
protected AsyncProcess ap; // non-final so can be overridden in test
|
||||
|
||||
BufferedMutatorImpl(ClusterConnection conn, RpcRetryingCallerFactory rpcCallerFactory,
|
||||
|
@ -200,7 +195,6 @@ public class BufferedMutatorImpl implements BufferedMutator {
|
|||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
protected long getExecutedWriteBufferPeriodicFlushes() {
|
||||
return executedWriteBufferPeriodicFlushes.get();
|
||||
}
|
||||
|
@ -425,7 +419,6 @@ public class BufferedMutatorImpl implements BufferedMutator {
|
|||
this.ap.setOperationTimeout(operationTimeout);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
long getCurrentWriteBufferSize() {
|
||||
return currentWriteBufferSize.get();
|
||||
}
|
||||
|
@ -440,12 +433,10 @@ public class BufferedMutatorImpl implements BufferedMutator {
|
|||
return Arrays.asList(writeAsyncBuffer.toArray(new Row[0]));
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
QueueRowAccess createQueueRowAccess() {
|
||||
return new QueueRowAccess();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
class QueueRowAccess implements RowAccess<Row>, Closeable {
|
||||
private int remainder = undealtMutationCount.getAndSet(0);
|
||||
private Mutation last = null;
|
||||
|
|
|
@ -19,8 +19,6 @@ package org.apache.hadoop.hbase.client;
|
|||
|
||||
import static org.apache.hadoop.hbase.client.ConnectionUtils.createScanResultCache;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.util.LinkedList;
|
||||
|
@ -186,7 +184,6 @@ public abstract class ClientScanner extends AbstractClientScanner {
|
|||
return lastNext;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
protected long getMaxResultSize() {
|
||||
return maxScannerResultSize;
|
||||
}
|
||||
|
@ -219,7 +216,6 @@ public abstract class ClientScanner extends AbstractClientScanner {
|
|||
* Marked as protected only because TestClientScanner need to override this method.
|
||||
* @return false if we should terminate the scan. Otherwise
|
||||
*/
|
||||
@VisibleForTesting
|
||||
protected boolean moveToNextRegion() {
|
||||
// Close the previous scanner if it's open
|
||||
try {
|
||||
|
@ -258,7 +254,6 @@ public abstract class ClientScanner extends AbstractClientScanner {
|
|||
return true;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
boolean isAnyRPCcancelled() {
|
||||
return callable.isAnyRPCcancelled();
|
||||
}
|
||||
|
@ -317,7 +312,6 @@ public abstract class ClientScanner extends AbstractClientScanner {
|
|||
return null;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public int getCacheSize() {
|
||||
return cache != null ? cache.size() : 0;
|
||||
}
|
||||
|
|
|
@ -11,7 +11,6 @@
|
|||
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -126,7 +125,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;
|
||||
|
|
|
@ -20,6 +20,9 @@ package org.apache.hadoop.hbase.client;
|
|||
|
||||
import static org.apache.hadoop.hbase.client.MetricsConnection.CLIENT_SIDE_METRICS_ENABLED_KEY;
|
||||
|
||||
import com.google.protobuf.BlockingRpcChannel;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
|
@ -45,7 +48,6 @@ import java.util.concurrent.TimeUnit;
|
|||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -202,17 +204,11 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.ExceptionUtil;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.protobuf.BlockingRpcChannel;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
/**
|
||||
* An internal, non-instantiable class that manages creation of {@link HConnection}s.
|
||||
*/
|
||||
|
@ -282,7 +278,6 @@ class ConnectionManager {
|
|||
* @param cnm Replaces the nonce generator used, for testing.
|
||||
* @return old nonce generator.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
static NonceGenerator injectNonceGeneratorForTesting(
|
||||
ClusterConnection conn, NonceGenerator cnm) {
|
||||
HConnectionImplementation connImpl = (HConnectionImplementation)conn;
|
||||
|
@ -752,7 +747,6 @@ class ConnectionManager {
|
|||
/**
|
||||
* @param useMetaReplicas
|
||||
*/
|
||||
@VisibleForTesting
|
||||
void setUseMetaReplicas(final boolean useMetaReplicas) {
|
||||
this.useMetaReplicas = useMetaReplicas;
|
||||
}
|
||||
|
@ -933,7 +927,6 @@ class ConnectionManager {
|
|||
/**
|
||||
* For tests only.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
RpcClient getRpcClient() {
|
||||
return rpcClient;
|
||||
}
|
||||
|
@ -2503,7 +2496,6 @@ class ConnectionManager {
|
|||
* Return the number of cached region for a table. It will only be called
|
||||
* from a unit test.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
int getNumberOfCachedRegionLocations(final TableName tableName) {
|
||||
return metaCache.getNumberOfCachedRegionLocations(tableName);
|
||||
}
|
||||
|
|
|
@ -20,8 +20,6 @@ package org.apache.hadoop.hbase.client;
|
|||
import static org.apache.hadoop.hbase.HConstants.EMPTY_END_ROW;
|
||||
import static org.apache.hadoop.hbase.HConstants.EMPTY_START_ROW;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
|
@ -169,7 +167,6 @@ public class ConnectionUtils {
|
|||
* Setup the connection class, so that it will not depend on master being online. Used for testing
|
||||
* @param conf configuration to set
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public static void setupMasterlessConnection(Configuration conf) {
|
||||
conf.set(HConnection.HBASE_CLIENT_CONNECTION_IMPL,
|
||||
MasterlessConnection.class.getName());
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
|
@ -2810,7 +2809,6 @@ public class HBaseAdmin implements Admin {
|
|||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public void split(final ServerName sn, final HRegionInfo hri,
|
||||
byte[] splitPoint) throws IOException {
|
||||
if (hri.getStartKey() != null && splitPoint != null &&
|
||||
|
|
|
@ -31,8 +31,6 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* An implementation of {@link RegionLocator}. Used to view region location information for a single
|
||||
* HBase table. Lightweight. Get as needed and just close when done. Instances of this class SHOULD
|
||||
|
@ -117,7 +115,6 @@ public class HRegionLocator implements RegionLocator {
|
|||
return getStartEndKeys(listRegionLocations());
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
Pair<byte[][], byte[][]> getStartEndKeys(List<RegionLocations> regions) {
|
||||
final byte[][] startKeyList = new byte[regions.size()][];
|
||||
final byte[][] endKeyList = new byte[regions.size()][];
|
||||
|
@ -136,7 +133,6 @@ public class HRegionLocator implements RegionLocator {
|
|||
return this.tableName;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
List<RegionLocations> listRegionLocations() throws IOException {
|
||||
return MetaScanner.listTableRegionLocations(getConfiguration(), this.connection, getName());
|
||||
}
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.protobuf.Descriptors;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.Service;
|
||||
|
@ -333,7 +332,6 @@ public class HTable implements HTableInterface, RegionLocator {
|
|||
* For internal testing. Uses Connection provided in {@code params}.
|
||||
* @throws IOException
|
||||
*/
|
||||
@VisibleForTesting
|
||||
protected HTable(ClusterConnection conn, BufferedMutatorParams params) throws IOException {
|
||||
connection = conn;
|
||||
tableName = params.getTableName();
|
||||
|
@ -536,7 +534,6 @@ public class HTable implements HTableInterface, RegionLocator {
|
|||
*/
|
||||
// TODO(tsuna): Remove this. Unit tests shouldn't require public helpers.
|
||||
@Deprecated
|
||||
@VisibleForTesting
|
||||
public HConnection getConnection() {
|
||||
return this.connection;
|
||||
}
|
||||
|
@ -1971,7 +1968,6 @@ public class HTable implements HTableInterface, RegionLocator {
|
|||
return this.locator;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
BufferedMutator getBufferedMutator() throws IOException {
|
||||
if (mutator == null) {
|
||||
this.mutator = (BufferedMutatorImpl) connection.getBufferedMutator(
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -242,7 +241,7 @@ public class HTableMultiplexer {
|
|||
return new HTableMultiplexerStatus(serverToFlushWorkerMap);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
LinkedBlockingQueue<PutStatus> getQueue(HRegionLocation addr) {
|
||||
FlushWorker worker = serverToFlushWorkerMap.get(addr);
|
||||
if (worker == null) {
|
||||
|
@ -260,7 +259,7 @@ public class HTableMultiplexer {
|
|||
return worker.getQueue();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
ClusterConnection getConnection() {
|
||||
return this.conn;
|
||||
}
|
||||
|
@ -373,7 +372,7 @@ public class HTableMultiplexer {
|
|||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
static class PutStatus {
|
||||
public final HRegionInfo regionInfo;
|
||||
public final Put put;
|
||||
|
@ -426,7 +425,7 @@ public class HTableMultiplexer {
|
|||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
static class FlushWorker implements Runnable {
|
||||
private final HRegionLocation addr;
|
||||
private final LinkedBlockingQueue<PutStatus> queue;
|
||||
|
@ -525,33 +524,33 @@ public class HTableMultiplexer {
|
|||
return true;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
long getNextDelay(int retryCount) {
|
||||
return ConnectionUtils.getPauseTime(multiplexer.flushPeriod,
|
||||
multiplexer.retryNum - retryCount - 1);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
AtomicInteger getRetryInQueue() {
|
||||
return this.retryInQueue;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
int getMaxRetryInQueue() {
|
||||
return this.maxRetryInQueue;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
AtomicLong getTotalFailedPutCount() {
|
||||
return this.totalFailedPutCount;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
HTableMultiplexer getMultiplexer() {
|
||||
return this.multiplexer;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
ScheduledExecutorService getExecutor() {
|
||||
return this.executor;
|
||||
}
|
||||
|
|
|
@ -41,8 +41,6 @@ import org.apache.hadoop.hbase.TableNotFoundException;
|
|||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.ExceptionUtil;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* Scanner class that contains the <code>hbase:meta</code> table scanning logic.
|
||||
* Provided visitors will be called for each row.
|
||||
|
@ -68,7 +66,7 @@ public class MetaScanner {
|
|||
* @param visitor A custom visitor
|
||||
* @throws IOException e
|
||||
*/
|
||||
@VisibleForTesting // Do not use. Used by tests only and hbck.
|
||||
// Do not use. Used by tests only and hbck.
|
||||
public static void metaScan(Connection connection,
|
||||
MetaScannerVisitor visitor) throws IOException {
|
||||
metaScan(connection, visitor, null, null, Integer.MAX_VALUE);
|
||||
|
@ -108,7 +106,7 @@ public class MetaScanner {
|
|||
* will be set to default value <code>Integer.MAX_VALUE</code>.
|
||||
* @throws IOException e
|
||||
*/
|
||||
@VisibleForTesting // Do not use. Used by Master but by a method that is used testing.
|
||||
// Do not use. Used by Master but by a method that is used testing.
|
||||
public static void metaScan(Connection connection,
|
||||
MetaScannerVisitor visitor, TableName userTableName, byte[] row,
|
||||
int rowLimit)
|
||||
|
@ -252,7 +250,7 @@ public class MetaScanner {
|
|||
* @return List of all user-space regions.
|
||||
* @throws IOException
|
||||
*/
|
||||
@VisibleForTesting // And for hbck.
|
||||
// And for hbck.
|
||||
public static List<HRegionInfo> listAllRegions(Configuration conf, Connection connection,
|
||||
final boolean offlined)
|
||||
throws IOException {
|
||||
|
|
|
@ -17,7 +17,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.protobuf.Descriptors.MethodDescriptor;
|
||||
import com.google.protobuf.Message;
|
||||
import com.yammer.metrics.core.Counter;
|
||||
|
@ -115,12 +114,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(MetricsRegistry registry, String name, String subName, String scope) {
|
||||
StringBuilder sb = new StringBuilder(CLIENT_SVC).append("_").append(name);
|
||||
|
@ -168,7 +166,6 @@ public class MetricsConnection implements StatisticTrackable {
|
|||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
protected static class RunnerStats {
|
||||
final Counter normalRunners;
|
||||
final Counter delayRunners;
|
||||
|
@ -193,7 +190,6 @@ public class MetricsConnection implements StatisticTrackable {
|
|||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
protected ConcurrentHashMap<ServerName, ConcurrentMap<byte[], RegionStats>> serverStats
|
||||
= new ConcurrentHashMap<ServerName, ConcurrentMap<byte[], RegionStats>>();
|
||||
|
||||
|
@ -276,35 +272,35 @@ 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;
|
||||
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;
|
||||
|
||||
// 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);
|
||||
|
||||
public MetricsConnection(final ConnectionManager.HConnectionImplementation conn) {
|
||||
|
@ -368,17 +364,14 @@ public class MetricsConnection implements StatisticTrackable {
|
|||
this.reporter.start();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
final MetricName getExecutorPoolName() {
|
||||
return new MetricName(getClass(), "executorPoolActiveThreads", scope);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
final MetricName getMetaPoolName() {
|
||||
return new MetricName(getClass(), "metaPoolActiveThreads", scope);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
MetricsRegistry getMetricsRegistry() {
|
||||
return registry;
|
||||
}
|
||||
|
|
|
@ -17,12 +17,12 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import com.google.protobuf.ServiceException;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.hbase.CellScannable;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
|
@ -41,9 +41,6 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
/**
|
||||
* Callable that handles the <code>multi</code> method call going against a single
|
||||
* regionserver; i.e. A {@link RegionServerCallable} for the multi call (It is not a
|
||||
|
@ -199,7 +196,6 @@ class MultiServerCallable<R> extends PayloadCarryingServerCallable<MultiResponse
|
|||
setStub(getConnection().getClient(this.location.getServerName()));
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
ServerName getServerName() {
|
||||
return location.getServerName();
|
||||
}
|
||||
|
|
|
@ -35,8 +35,6 @@ import org.apache.hadoop.hbase.ipc.CallTimeoutException;
|
|||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
*
|
||||
* The concrete {@link RetryingCallerInterceptor} class that implements the preemptive fast fail
|
||||
|
@ -149,7 +147,6 @@ class PreemptiveFastFailInterceptor extends RetryingCallerInterceptor {
|
|||
* - the throwable to be handled.
|
||||
* @throws PreemptiveFastFailException
|
||||
*/
|
||||
@VisibleForTesting
|
||||
protected void handleFailureToServer(ServerName serverName, Throwable t) {
|
||||
if (serverName == null || t == null) {
|
||||
return;
|
||||
|
|
|
@ -17,8 +17,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
|
||||
|
@ -30,7 +28,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
|
|||
*/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
@VisibleForTesting
|
||||
interface RowAccess<T> extends Iterable<T> {
|
||||
/**
|
||||
* @return true if there are no elements.
|
||||
|
|
|
@ -18,8 +18,6 @@
|
|||
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.util.HashSet;
|
||||
|
@ -363,7 +361,6 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
|
|||
callable.getScan().withStartRow(this.lastResult.getRow(), this.lastResult.mayHaveMoreCellsInRow());
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
boolean isAnyRPCcancelled() {
|
||||
return someRPCcancelled;
|
||||
}
|
||||
|
|
|
@ -17,7 +17,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
|
@ -25,9 +25,6 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.hbase.client.backoff.ServerStatistics;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
/**
|
||||
* Tracks the statistics for multiple regions
|
||||
*/
|
||||
|
@ -69,7 +66,6 @@ public class ServerStatisticTracker implements StatisticTrackable {
|
|||
return new ServerStatisticTracker();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
ServerStatistics getServerStatsForTesting(ServerName server) {
|
||||
return stats.get(server);
|
||||
}
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.client.replication;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -29,8 +30,6 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.commons.lang.StringUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -53,9 +52,6 @@ import org.apache.hadoop.hbase.replication.ReplicationSerDeHelper;
|
|||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* This class provides the administrative interface to HBase cluster
|
||||
|
@ -223,7 +219,7 @@ public class ReplicationAdmin implements Closeable {
|
|||
return ReplicationSerDeHelper.parseTableCFsFromConfig(tableCFsConfig);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
static String getTableCfsStr(Map<TableName, ? extends Collection<String>> tableCfs) {
|
||||
String tableCfsStr = null;
|
||||
if (tableCfs != null) {
|
||||
|
@ -608,12 +604,12 @@ public class ReplicationAdmin implements Closeable {
|
|||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
public void peerAdded(String id) throws ReplicationException {
|
||||
this.replicationPeers.peerAdded(id);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
List<ReplicationPeer> listReplicationPeers() {
|
||||
Map<String, ReplicationPeerConfig> peers = listPeerConfigs();
|
||||
if (peers == null || peers.size() <= 0) {
|
||||
|
|
|
@ -17,13 +17,13 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.filter;
|
||||
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.PriorityQueue;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
@ -36,9 +36,6 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.util.UnsafeAvailChecker;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
|
||||
/**
|
||||
* 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.
|
||||
|
@ -318,12 +315,12 @@ public class FuzzyRowFilter extends FilterBase {
|
|||
NO_NEXT
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
static SatisfiesCode satisfies(byte[] row, byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) {
|
||||
return satisfies(false, row, 0, row.length, fuzzyKeyBytes, fuzzyKeyMeta);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
static SatisfiesCode satisfies(boolean reverse, byte[] row, byte[] fuzzyKeyBytes,
|
||||
byte[] fuzzyKeyMeta) {
|
||||
return satisfies(reverse, row, 0, row.length, fuzzyKeyBytes, fuzzyKeyMeta);
|
||||
|
@ -439,12 +436,12 @@ public class FuzzyRowFilter extends FilterBase {
|
|||
return SatisfiesCode.YES;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
static byte[] getNextForFuzzyRule(byte[] row, byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) {
|
||||
return getNextForFuzzyRule(false, row, 0, row.length, fuzzyKeyBytes, fuzzyKeyMeta);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
static byte[] getNextForFuzzyRule(boolean reverse, byte[] row, byte[] fuzzyKeyBytes,
|
||||
byte[] fuzzyKeyMeta) {
|
||||
return getNextForFuzzyRule(reverse, row, 0, row.length, fuzzyKeyBytes, fuzzyKeyMeta);
|
||||
|
@ -531,7 +528,7 @@ public class FuzzyRowFilter extends FilterBase {
|
|||
* @return greater byte array than given (row) which satisfies the fuzzy rule if it exists, null
|
||||
* otherwise
|
||||
*/
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
static byte[] getNextForFuzzyRule(boolean reverse, byte[] row, int offset, int length,
|
||||
byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) {
|
||||
// To find out the next "smallest" byte array that satisfies fuzzy rule and "greater" than
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.ipc;
|
|||
import static org.apache.hadoop.hbase.ipc.IPCUtil.toIOE;
|
||||
import static org.apache.hadoop.hbase.ipc.IPCUtil.wrapException;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.cache.CacheBuilder;
|
||||
import com.google.common.cache.CacheLoader;
|
||||
|
@ -222,7 +221,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
|
||||
|
@ -254,7 +252,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;
|
||||
}
|
||||
|
@ -556,7 +553,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 {
|
||||
|
||||
|
|
|
@ -17,8 +17,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.ipc;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.SocketAddress;
|
||||
|
||||
|
@ -44,7 +42,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);
|
||||
}
|
||||
|
|
|
@ -17,7 +17,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.ipc;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
|
||||
import java.net.SocketAddress;
|
||||
|
@ -46,7 +45,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);
|
||||
}
|
||||
|
|
|
@ -10,8 +10,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.quotas;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
|
@ -56,7 +54,6 @@ class ThrottleSettings extends QuotaSettings {
|
|||
builder.setThrottle(proto);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
ThrottleRequest getProto() {
|
||||
return proto;
|
||||
}
|
||||
|
|
|
@ -20,12 +20,9 @@ package org.apache.hadoop.hbase.replication;
|
|||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.SortedMap;
|
||||
import java.util.SortedSet;
|
||||
import java.util.TreeMap;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
@ -300,7 +297,6 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
|
|||
* @param znode the server names of the other server
|
||||
* @return true if the lock was acquired, false in every other cases
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public boolean lockOtherRS(String znode) {
|
||||
try {
|
||||
String parent = ZKUtil.joinZNode(this.queuesZNode, znode);
|
||||
|
@ -332,7 +328,6 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
|
|||
return this.queuesZNode + "/" + znode + "/" + RS_LOCK_ZNODE;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public boolean checkLockExists(String znode) throws KeeperException {
|
||||
return ZKUtil.checkExists(zookeeper, getLockZNode(znode)) >= 0;
|
||||
}
|
||||
|
|
|
@ -30,9 +30,6 @@ import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
|||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
|
||||
/**
|
||||
* This is a base class for maintaining replication state in zookeeper.
|
||||
*/
|
||||
|
@ -131,17 +128,14 @@ public abstract class ReplicationStateZKBase {
|
|||
return path.split("/").length == peersZNode.split("/").length + 1;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
protected String getTableCFsNode(String id) {
|
||||
return ZKUtil.joinZNode(this.peersZNode, ZKUtil.joinZNode(id, this.tableCFsNodeName));
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
protected String getPeerStateNode(String id) {
|
||||
return ZKUtil.joinZNode(this.peersZNode, ZKUtil.joinZNode(id, this.peerStateNodeName));
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
protected String getPeerNode(String id) {
|
||||
return ZKUtil.joinZNode(this.peersZNode, id);
|
||||
}
|
||||
|
|
|
@ -17,8 +17,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.security;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -146,7 +144,6 @@ public abstract class AbstractHBaseSaslRpcClient {
|
|||
SaslUtil.safeDispose(saslClient);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
static class SaslClientCallbackHandler implements CallbackHandler {
|
||||
private final String userName;
|
||||
private final char[] userPassword;
|
||||
|
|
|
@ -27,7 +27,6 @@ import java.util.concurrent.ScheduledThreadPoolExecutor;
|
|||
import java.util.concurrent.ThreadFactory;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.ScheduledChore.ChoreServicer;
|
||||
|
@ -96,7 +95,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);
|
||||
}
|
||||
|
|
|
@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.io.util.StreamUtils;
|
|||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.ClassSize;
|
||||
import org.apache.hadoop.io.RawComparator;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* An HBase Key/Value. This is the fundamental HBase Type.
|
||||
|
@ -2655,7 +2654,6 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
|
|||
}
|
||||
|
||||
@Override
|
||||
@VisibleForTesting
|
||||
public int compareOnlyKeyPortion(Cell left, Cell right) {
|
||||
int c = Bytes.BYTES_RAWCOMPARATOR.compare(left.getRowArray(), left.getRowOffset(),
|
||||
left.getRowLength(), right.getRowArray(), right.getRowOffset(), right.getRowLength());
|
||||
|
|
|
@ -26,8 +26,6 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
|
||||
import 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
|
||||
|
@ -119,7 +117,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.name = null;
|
||||
this.stopper = null;
|
||||
|
@ -309,17 +306,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;
|
||||
}
|
||||
|
@ -332,7 +329,6 @@ public abstract class ScheduledChore implements Runnable {
|
|||
}
|
||||
|
||||
@InterfaceAudience.Private
|
||||
@VisibleForTesting
|
||||
public synchronized void choreForTesting() {
|
||||
chore();
|
||||
}
|
||||
|
|
|
@ -27,8 +27,6 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.util.BoundedArrayQueue;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* Like Hadoops' ByteBufferPool only you do not specify desired size when getting a ByteBuffer.
|
||||
* This pool keeps an upper bound on the count of ByteBuffers in the pool and on the maximum size
|
||||
|
@ -50,14 +48,12 @@ import com.google.common.annotations.VisibleForTesting;
|
|||
public class BoundedByteBufferPool {
|
||||
private static final Log LOG = LogFactory.getLog(BoundedByteBufferPool.class);
|
||||
|
||||
@VisibleForTesting
|
||||
final Queue<ByteBuffer> buffers;
|
||||
|
||||
// Maximum size of a ByteBuffer to retain in pool
|
||||
private final int maxByteBufferSizeToCache;
|
||||
|
||||
// A running average only it only rises, it never recedes
|
||||
@VisibleForTesting
|
||||
volatile int runningAverage;
|
||||
|
||||
// Scratch that keeps rough total size of pooled bytebuffers
|
||||
|
|
|
@ -17,15 +17,14 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.io.crypto.aes;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.security.GeneralSecurityException;
|
||||
import java.security.Key;
|
||||
import java.security.SecureRandom;
|
||||
|
||||
import javax.crypto.spec.SecretKeySpec;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
@ -36,9 +35,6 @@ import org.apache.hadoop.hbase.io.crypto.Context;
|
|||
import org.apache.hadoop.hbase.io.crypto.Decryptor;
|
||||
import org.apache.hadoop.hbase.io.crypto.Encryptor;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* AES-128, provided by the JCE
|
||||
* <p>
|
||||
|
@ -156,7 +152,6 @@ public class AES extends Cipher {
|
|||
return d.createDecryptionStream(in);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
SecureRandom getRNG() {
|
||||
return rng;
|
||||
}
|
||||
|
|
|
@ -16,6 +16,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.io.encoding;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.DataInputStream;
|
||||
|
@ -24,7 +25,6 @@ import java.io.IOException;
|
|||
import java.io.OutputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Iterator;
|
||||
|
||||
import org.apache.commons.lang.NotImplementedException;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
|
@ -37,16 +37,12 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.io.compress.Compressor;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* Encapsulates a data block compressed using a particular encoding algorithm.
|
||||
* Useful for testing and benchmarking.
|
||||
* This is used only in testing.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@VisibleForTesting
|
||||
public class EncodedDataBlock {
|
||||
private byte[] rawKVs;
|
||||
private ByteBuffer rawBuffer;
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.util;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.concurrent.Callable;
|
||||
|
@ -44,10 +43,8 @@ public class ByteBufferArray {
|
|||
private static final Log LOG = LogFactory.getLog(ByteBufferArray.class);
|
||||
|
||||
static final int DEFAULT_BUFFER_SIZE = 4 * 1024 * 1024;
|
||||
@VisibleForTesting
|
||||
ByteBuffer[] buffers;
|
||||
private int bufferSize;
|
||||
@VisibleForTesting
|
||||
int bufferCount;
|
||||
|
||||
/**
|
||||
|
@ -70,7 +67,6 @@ public class ByteBufferArray {
|
|||
createBuffers(directByteBuffer, allocator);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
void createBuffers(boolean directByteBuffer, ByteBufferAllocator allocator)
|
||||
throws IOException {
|
||||
int threadCount = getThreadCount();
|
||||
|
@ -105,7 +101,6 @@ public class ByteBufferArray {
|
|||
this.buffers[bufferCount] = ByteBuffer.allocate(0);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
int getThreadCount() {
|
||||
return Runtime.getRuntime().availableProcessors();
|
||||
}
|
||||
|
|
|
@ -29,8 +29,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
|
|||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.io.WritableUtils;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
import sun.nio.ch.DirectBuffer;
|
||||
|
||||
/**
|
||||
|
@ -47,7 +45,7 @@ public final class ByteBufferUtils {
|
|||
public final static int VALUE_MASK = 0x7f;
|
||||
public final static int NEXT_BIT_SHIFT = 7;
|
||||
public final static int NEXT_BIT_MASK = 1 << 7;
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
static final boolean UNSAFE_AVAIL = UnsafeAvailChecker.isAvailable();
|
||||
public static final boolean UNSAFE_UNALIGNED = UnsafeAvailChecker.unaligned();
|
||||
|
||||
|
|
|
@ -21,6 +21,7 @@ import static com.google.common.base.Preconditions.checkArgument;
|
|||
import static com.google.common.base.Preconditions.checkNotNull;
|
||||
import static com.google.common.base.Preconditions.checkPositionIndex;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
|
@ -35,24 +36,18 @@ import java.util.Collection;
|
|||
import java.util.Comparator;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.util.Bytes.LexicographicalComparerHolder.UnsafeComparer;
|
||||
import org.apache.hadoop.io.RawComparator;
|
||||
import org.apache.hadoop.io.WritableComparator;
|
||||
import org.apache.hadoop.io.WritableUtils;
|
||||
|
||||
import sun.misc.Unsafe;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
import org.apache.hadoop.hbase.util.Bytes.LexicographicalComparerHolder.UnsafeComparer;
|
||||
|
||||
/**
|
||||
* Utility class that handles byte arrays, conversions to/from other types,
|
||||
* comparisons, hash code generation, manufacturing keys for HashMaps or
|
||||
|
@ -129,7 +124,7 @@ public class Bytes implements Comparable<Bytes> {
|
|||
// SizeOf which uses java.lang.instrument says 24 bytes. (3 longs?)
|
||||
public static final int ESTIMATED_HEAP_TAX = 16;
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
static final boolean UNSAFE_UNALIGNED = UnsafeAvailChecker.unaligned();
|
||||
|
||||
/**
|
||||
|
@ -1368,7 +1363,7 @@ public class Bytes implements Comparable<Bytes> {
|
|||
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
static Comparer<byte[]> lexicographicalComparerJavaImpl() {
|
||||
return LexicographicalComparerHolder.PureJavaComparer.INSTANCE;
|
||||
}
|
||||
|
@ -1518,7 +1513,7 @@ public class Bytes implements Comparable<Bytes> {
|
|||
* <p>Uses reflection to gracefully fall back to the Java implementation if
|
||||
* {@code Unsafe} isn't available.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
static class LexicographicalComparerHolder {
|
||||
static final String UNSAFE_COMPARER_NAME =
|
||||
LexicographicalComparerHolder.class.getName() + "$UnsafeComparer";
|
||||
|
@ -1568,7 +1563,7 @@ public class Bytes implements Comparable<Bytes> {
|
|||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
enum UnsafeComparer implements Comparer<byte[]> {
|
||||
INSTANCE;
|
||||
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.util;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
|
@ -338,7 +337,6 @@ public abstract class CommonFSUtils {
|
|||
return p.makeQualified(fs);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public static void setWALRootDir(final Configuration c, final Path root) {
|
||||
c.set(HBASE_WAL_DIR, root.toString());
|
||||
}
|
||||
|
|
|
@ -29,8 +29,6 @@ import java.nio.charset.Charset;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
|
||||
import 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
|
||||
|
@ -370,7 +368,7 @@ public class OrderedBytes {
|
|||
* @param comp Compliment the encoded value when {@code comp} is true.
|
||||
* @return number of bytes written.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
static int putVaruint64(PositionedByteRange dst, long val, boolean comp) {
|
||||
int w, y, len = 0;
|
||||
final int offset = dst.getOffset(), start = dst.getPosition();
|
||||
|
@ -459,7 +457,7 @@ public class OrderedBytes {
|
|||
* @param comp if true, parse the compliment of the value.
|
||||
* @return the number of bytes consumed by this value.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
static int lengthVaruint64(PositionedByteRange src, boolean comp) {
|
||||
int a0 = (comp ? DESCENDING : ASCENDING).apply(src.peek()) & 0xff;
|
||||
if (a0 <= 240) return 1;
|
||||
|
@ -480,7 +478,7 @@ public class OrderedBytes {
|
|||
* @param cmp if true, parse the compliment of the value.
|
||||
* @return the number of bytes skipped.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
static int skipVaruint64(PositionedByteRange src, boolean cmp) {
|
||||
final int len = lengthVaruint64(src, cmp);
|
||||
src.setPosition(src.getPosition() + len);
|
||||
|
@ -492,7 +490,7 @@ public class OrderedBytes {
|
|||
* encoded value when {@code comp} is true.
|
||||
* @return the decoded value.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
static long getVaruint64(PositionedByteRange src, boolean comp) {
|
||||
assert src.getRemaining() >= lengthVaruint64(src, comp);
|
||||
final long ret;
|
||||
|
@ -549,7 +547,7 @@ public class OrderedBytes {
|
|||
* From Phoenix's {@code NumberUtil}.
|
||||
* @return new {@link BigDecimal} instance
|
||||
*/
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
static BigDecimal normalize(BigDecimal val) {
|
||||
return null == val ? null : val.stripTrailingZeros().round(DEFAULT_MATH_CONTEXT);
|
||||
}
|
||||
|
@ -1015,7 +1013,7 @@ public class OrderedBytes {
|
|||
/**
|
||||
* Calculate the expected BlobVar decoded length based on encoded length.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
static int blobVarDecodedLength(int len) {
|
||||
return
|
||||
((len
|
||||
|
|
|
@ -25,7 +25,6 @@ import java.util.List;
|
|||
import java.util.Map.Entry;
|
||||
import java.util.Properties;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -466,7 +465,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(",");
|
||||
|
@ -479,7 +477,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;
|
||||
|
|
|
@ -17,7 +17,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.metrics.impl;
|
||||
|
||||
|
||||
import com.google.common.base.Optional;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.lang.reflect.Method;
|
||||
import java.util.Collection;
|
||||
|
@ -27,7 +27,6 @@ import java.util.Map;
|
|||
import java.util.Map.Entry;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import org.apache.commons.lang.reflect.FieldUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -44,9 +43,6 @@ import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
|||
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper;
|
||||
import org.apache.hadoop.metrics2.lib.MetricsExecutorImpl;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Optional;
|
||||
|
||||
/**
|
||||
* 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
|
||||
|
@ -113,7 +109,6 @@ public class GlobalMetricRegistriesAdapter {
|
|||
return new GlobalMetricRegistriesAdapter();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public void stop() {
|
||||
stopped.set(true);
|
||||
}
|
||||
|
@ -185,7 +180,6 @@ public class GlobalMetricRegistriesAdapter {
|
|||
* Use reflection to unregister the Hadoop metric source, since MetricsSystem#unregisterSource()
|
||||
* is only available in Hadoop 2.6+ (HADOOP-10839)
|
||||
*/
|
||||
@VisibleForTesting
|
||||
protected void unregisterSource(MetricRegistryInfo info) {
|
||||
// unregisterSource is only available in Hadoop 2.6+ (HADOOP-10839). Don't unregister for now
|
||||
MetricsSystem metricsSystem = DefaultMetricsSystem.instance();
|
||||
|
|
|
@ -26,8 +26,6 @@ import org.apache.hadoop.metrics2.MetricsCollector;
|
|||
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
|
||||
import org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* Implementation of {@link MetricsTableLatencies} to track latencies for one table in a
|
||||
* RegionServer.
|
||||
|
@ -37,7 +35,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;
|
||||
|
@ -115,7 +112,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());
|
||||
|
@ -124,7 +120,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);
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
|
||||
|
@ -88,7 +87,6 @@ public class MetricsUserAggregateSourceImpl extends BaseSourceImpl
|
|||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public ConcurrentHashMap<String, MetricsUserSource> getUserSources() {
|
||||
return userSources;
|
||||
}
|
||||
|
|
|
@ -30,8 +30,6 @@ import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
|||
import org.apache.hadoop.metrics2.lib.MetricsExecutorImpl;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
|
||||
import 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 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 class JmxCacheBuster {
|
|||
* Restarts the stopped service.
|
||||
* @see #stop()
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public static void restart() {
|
||||
stopped.set(false);
|
||||
}
|
||||
|
|
|
@ -27,8 +27,6 @@ import java.util.Map;
|
|||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
||||
import 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();
|
||||
}
|
||||
|
|
|
@ -30,8 +30,6 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class MetricRegistriesLoader {
|
||||
private static final Log LOG = LogFactory.getLog(MetricRegistries.class);
|
||||
|
@ -56,7 +54,6 @@ public class MetricRegistriesLoader {
|
|||
* implementation will be loaded.
|
||||
* @return A {@link MetricRegistries} implementation.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
static MetricRegistries load(List<MetricRegistries> availableImplementations) {
|
||||
|
||||
if (availableImplementations.size() == 1) {
|
||||
|
|
|
@ -18,6 +18,8 @@
|
|||
|
||||
package org.apache.hadoop.hbase.procedure2;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.protobuf.ByteString;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
|
@ -26,7 +28,6 @@ import java.lang.reflect.Modifier;
|
|||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.ProcedureInfo;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
@ -39,10 +40,6 @@ import org.apache.hadoop.hbase.util.ByteStringer;
|
|||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.NonceKey;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.protobuf.ByteString;
|
||||
|
||||
/**
|
||||
* Base Procedure class responsible to handle the Procedure Metadata
|
||||
* e.g. state, startTime, lastUpdate, stack-indexes, ...
|
||||
|
@ -414,7 +411,6 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
|
|||
return Math.max(0, timeout - (EnvironmentEdgeManager.currentTime() - startTime));
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
public void setOwner(final String owner) {
|
||||
this.owner = StringUtils.isEmpty(owner) ? null : owner;
|
||||
|
@ -428,7 +424,6 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
|
|||
return owner != null;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
protected synchronized void setState(final ProcedureState state) {
|
||||
this.state = state;
|
||||
|
@ -469,7 +464,6 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
|
|||
/**
|
||||
* Called by the ProcedureExecutor to assign the ID to the newly created procedure.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
protected void setProcId(final long procId) {
|
||||
this.procId = procId;
|
||||
|
@ -488,7 +482,6 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
|
|||
/**
|
||||
* Called by the ProcedureExecutor to set the value to the newly created procedure.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
protected void setNonceKey(final NonceKey nonceKey) {
|
||||
this.nonceKey = nonceKey;
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
|
||||
package org.apache.hadoop.hbase.procedure2;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -1384,7 +1383,6 @@ public class ProcedureExecutor<TEnvironment> {
|
|||
return procId;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
protected long getLastProcId() {
|
||||
return lastProcId.get();
|
||||
}
|
||||
|
|
|
@ -18,8 +18,6 @@
|
|||
|
||||
package org.apache.hadoop.hbase.procedure2.store.wal;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -774,7 +772,6 @@ public class WALProcedureStore extends ProcedureStoreBase {
|
|||
return (System.currentTimeMillis() - lastRollTs.get());
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
protected void periodicRollForTesting() throws IOException {
|
||||
lock.lock();
|
||||
try {
|
||||
|
@ -784,7 +781,6 @@ public class WALProcedureStore extends ProcedureStoreBase {
|
|||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
protected boolean rollWriterForTesting() throws IOException {
|
||||
lock.lock();
|
||||
try {
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
|
||||
package org.apache.hadoop.hbase.rsgroup;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.collect.ArrayListMultimap;
|
||||
import com.google.common.collect.LinkedListMultimap;
|
||||
import com.google.common.collect.ListMultimap;
|
||||
|
@ -337,7 +336,6 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer, LoadBalanc
|
|||
return finalList;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public Set<HRegionInfo> getMisplacedRegions(
|
||||
Map<HRegionInfo, ServerName> regions) throws IOException {
|
||||
Set<HRegionInfo> misplacedRegions = new HashSet<HRegionInfo>();
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
|
||||
package org.apache.hadoop.hbase.rsgroup;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import java.util.Arrays;
|
||||
import org.apache.commons.cli.BasicParser;
|
||||
import org.apache.commons.cli.CommandLine;
|
||||
|
@ -47,7 +46,7 @@ public class RSGroupMajorCompactionTTL extends MajorCompactorTTL {
|
|||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(RSGroupMajorCompactionTTL.class);
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
RSGroupMajorCompactionTTL() {
|
||||
super();
|
||||
}
|
||||
|
|
|
@ -17,7 +17,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import java.util.Map;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorService;
|
||||
|
@ -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);
|
||||
|
|
|
@ -17,16 +17,14 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
/**
|
||||
* Class represents table state on HDFS.
|
||||
*/
|
||||
|
@ -50,7 +48,6 @@ public class TableDescriptor {
|
|||
* Creates TableDescriptor with Enabled table.
|
||||
* @param hTableDescriptor HTableDescriptor to use
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public TableDescriptor(HTableDescriptor hTableDescriptor) {
|
||||
this(hTableDescriptor, TableState.State.ENABLED);
|
||||
}
|
||||
|
|
|
@ -32,8 +32,6 @@ import org.apache.hadoop.hbase.master.SplitLogManager.ResubmitDirective;
|
|||
import org.apache.hadoop.hbase.master.SplitLogManager.Task;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
|
||||
import 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
|
||||
|
@ -215,6 +213,5 @@ public interface SplitLogManagerCoordination {
|
|||
* Support method to init constants such as timeout. Mostly required for UTs.
|
||||
* @throws IOException
|
||||
*/
|
||||
@VisibleForTesting
|
||||
void init() throws IOException;
|
||||
}
|
||||
|
|
|
@ -30,8 +30,6 @@ import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
|||
import org.apache.hadoop.hbase.regionserver.SplitLogWorker;
|
||||
import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* Coordinated operations for {@link SplitLogWorker} and
|
||||
* {@link org.apache.hadoop.hbase.regionserver.handler.WALSplitterHandler} Important
|
||||
|
@ -99,7 +97,6 @@ public interface SplitLogWorkerCoordination {
|
|||
* Used by unit tests to check how many tasks were processed
|
||||
* @return number of tasks
|
||||
*/
|
||||
@VisibleForTesting
|
||||
int getTaskReadySeq();
|
||||
|
||||
/**
|
||||
|
|
|
@ -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 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),
|
||||
|
|
|
@ -18,6 +18,9 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.executor;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
import java.io.IOException;
|
||||
import java.io.Writer;
|
||||
import java.lang.management.ThreadInfo;
|
||||
|
@ -31,18 +34,12 @@ import java.util.concurrent.LinkedBlockingQueue;
|
|||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.executor.EventHandler.EventHandlerListener;
|
||||
import org.apache.hadoop.hbase.monitoring.ThreadMonitoring;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
|
||||
/**
|
||||
* This is a generic executor service. This component abstracts a
|
||||
* threadpool, a queue to which {@link EventType}s can be submitted,
|
||||
|
@ -87,7 +84,6 @@ public class ExecutorService {
|
|||
* started with the same name, this throws a RuntimeException.
|
||||
* @param name Name of the service to start.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public void startExecutorService(String name, int maxThreads) {
|
||||
if (this.executorMap.get(name) != null) {
|
||||
throw new RuntimeException("An executor service with the name " + name +
|
||||
|
@ -127,7 +123,6 @@ public class ExecutorService {
|
|||
return executor;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public ThreadPoolExecutor getExecutorThreadPool(final ExecutorType type) {
|
||||
return getExecutor(type).getThreadPoolExecutor();
|
||||
}
|
||||
|
|
|
@ -31,7 +31,6 @@ import java.lang.reflect.Proxy;
|
|||
import java.lang.reflect.UndeclaredThrowableException;
|
||||
import java.net.URI;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -56,8 +55,6 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
|
|||
import org.apache.hadoop.ipc.RPC;
|
||||
import org.apache.hadoop.util.Progressable;
|
||||
|
||||
import static org.apache.hadoop.hbase.HConstants.HBASE_DIR;
|
||||
|
||||
/**
|
||||
* An encapsulation for the FileSystem object that hbase uses to access
|
||||
* data. This class allows the flexibility of using
|
||||
|
|
|
@ -17,7 +17,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.http.log;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Charsets;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
|
@ -89,7 +88,7 @@ public class LogLevel {
|
|||
return ((protocol.equals(PROTOCOL_HTTP) || protocol.equals(PROTOCOL_HTTPS)));
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
static class CLI extends Configured implements Tool {
|
||||
private Operations operation = Operations.UNKNOWN;
|
||||
private String protocol;
|
||||
|
|
|
@ -28,17 +28,16 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.fs.HFileSystem;
|
||||
import org.apache.hadoop.hdfs.DFSInputStream;
|
||||
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
|
||||
|
||||
import 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;
|
||||
* see method comments.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class FSDataInputStreamWrapper {
|
||||
private static final Log LOG = LogFactory.getLog(FSDataInputStreamWrapper.class);
|
||||
private static final boolean isLogTraceEnabled = LOG.isTraceEnabled();
|
||||
|
@ -160,13 +159,11 @@ public class FSDataInputStreamWrapper {
|
|||
}
|
||||
|
||||
/** 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;
|
||||
|
|
|
@ -19,8 +19,8 @@
|
|||
package org.apache.hadoop.hbase.io;
|
||||
|
||||
import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.regionserver.MetricsRegionServerSourceFactory;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
public class MetricsIO {
|
||||
|
||||
|
@ -37,12 +37,12 @@ public class MetricsIO {
|
|||
this.wrapper = wrapper;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
public MetricsIOSource getMetricsSource() {
|
||||
return source;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
public MetricsIOWrapper getWrapper() {
|
||||
return wrapper;
|
||||
}
|
||||
|
|
|
@ -36,8 +36,6 @@ import org.apache.hadoop.hbase.io.util.HeapMemorySizeUtil;
|
|||
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* Stores all of the cache objects and configuration for a single HFile.
|
||||
*/
|
||||
|
@ -440,7 +438,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;
|
||||
}
|
||||
|
@ -450,7 +447,6 @@ public class CacheConfig {
|
|||
* @param cacheDataInL1 Whether to cache data blocks up in l1 (if a multi-tier cache
|
||||
* implementation).
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public void setCacheDataInL1(boolean cacheDataInL1) {
|
||||
this.cacheDataInL1 = cacheDataInL1;
|
||||
}
|
||||
|
@ -612,13 +608,11 @@ public class CacheConfig {
|
|||
* at all.
|
||||
*/
|
||||
// Clear this if in tests you'd make more than one block cache instance.
|
||||
@VisibleForTesting
|
||||
static BlockCache GLOBAL_BLOCK_CACHE_INSTANCE;
|
||||
private static LruBlockCache GLOBAL_L1_CACHE_INSTANCE = null;
|
||||
private static BlockCache GLOBAL_L2_CACHE_INSTANCE = null;
|
||||
|
||||
/** Boolean whether we have disabled the block cache entirely. */
|
||||
@VisibleForTesting
|
||||
static boolean blockCacheDisabled = false;
|
||||
|
||||
/**
|
||||
|
@ -644,7 +638,6 @@ public class CacheConfig {
|
|||
* @return Returns L2 block cache instance (for now it is BucketCache BlockCache all the time)
|
||||
* or null if not supposed to be a L2.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
static BlockCache getL2(final Configuration c) {
|
||||
final boolean useExternal = c.getBoolean(EXTERNAL_BLOCKCACHE_KEY, EXTERNAL_BLOCKCACHE_DEFAULT);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
|
@ -797,7 +790,6 @@ public class CacheConfig {
|
|||
return GLOBAL_BLOCK_CACHE_INSTANCE;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
static synchronized void clearGlobalInstances() {
|
||||
GLOBAL_L1_CACHE_INSTANCE = null;
|
||||
GLOBAL_L2_CACHE_INSTANCE = null;
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.io.hfile;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import java.io.DataInputStream;
|
||||
import java.io.DataOutput;
|
||||
import java.io.DataOutputStream;
|
||||
|
@ -26,7 +27,6 @@ import java.nio.ByteBuffer;
|
|||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.concurrent.locks.Lock;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
|
@ -50,9 +50,6 @@ import org.apache.hadoop.hbase.util.ChecksumType;
|
|||
import org.apache.hadoop.hbase.util.ClassSize;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* Reads {@link HFile} version 2 blocks to HFiles and via {@link Cacheable} Interface to caches.
|
||||
* Version 2 was introduced in hbase-0.92.0. No longer has support for version 1 blocks since
|
||||
|
@ -320,7 +317,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, ByteBuffer b, boolean fillHeader,
|
||||
long offset, final int nextBlockOnDiskSize, int onDiskDataSizeWithHeader,
|
||||
|
@ -526,7 +522,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
|
||||
ByteBuffer dup = this.buf.duplicate();
|
||||
|
@ -771,7 +766,6 @@ public class HFileBlock implements Cacheable {
|
|||
* was successful
|
||||
* @throws IOException if failed to read the necessary bytes
|
||||
*/
|
||||
@VisibleForTesting
|
||||
static boolean positionalReadWithExtra(FSDataInputStream in,
|
||||
long position, byte[] buf, int bufOffset, int necessaryLen, int extraLen)
|
||||
throws IOException {
|
||||
|
@ -1219,7 +1213,6 @@ public class HFileBlock implements Cacheable {
|
|||
*
|
||||
* @return Returns a copy of uncompressed block bytes for caching on write
|
||||
*/
|
||||
@VisibleForTesting
|
||||
ByteBuffer cloneUncompressedBufferWithHeader() {
|
||||
expectState(State.BLOCK_READY);
|
||||
byte[] uncompressedBlockBytesWithHeader = baosInMemory.toByteArray();
|
||||
|
@ -2071,7 +2064,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(ByteBuffer buf) throws IOException {
|
||||
byte[] magicBuf = new byte[Math.min(buf.limit() - buf.position(), BlockType.MAGIC_LENGTH)];
|
||||
buf.get(magicBuf);
|
||||
|
|
|
@ -47,8 +47,6 @@ import org.apache.hadoop.io.WritableUtils;
|
|||
import org.apache.htrace.Trace;
|
||||
import org.apache.htrace.TraceScope;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* {@link HFile} reader for version 2.
|
||||
*/
|
||||
|
@ -1425,7 +1423,6 @@ public class HFileReaderV2 extends AbstractHFileReader {
|
|||
* Returns false if block prefetching was requested for this file and has
|
||||
* not completed, true otherwise
|
||||
*/
|
||||
@VisibleForTesting
|
||||
boolean prefetchComplete() {
|
||||
return PrefetchExecutor.isCompleted(path);
|
||||
}
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.io.hfile;
|
||||
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
import java.lang.ref.WeakReference;
|
||||
import java.util.EnumMap;
|
||||
import java.util.Iterator;
|
||||
|
@ -32,7 +33,6 @@ import java.util.concurrent.ScheduledExecutorService;
|
|||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
import com.google.common.base.Objects;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -46,9 +46,6 @@ import org.apache.hadoop.hbase.util.ClassSize;
|
|||
import org.apache.hadoop.hbase.util.HasThread;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
|
||||
/**
|
||||
* A block cache implementation that is memory-aware using {@link HeapSize},
|
||||
* memory-bound using an LRU eviction algorithm, and concurrent: backed by a
|
||||
|
@ -579,12 +576,10 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
|
|||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
boolean isEvictionInProgress() {
|
||||
return evictionInProgress;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
long getOverhead() {
|
||||
return overhead;
|
||||
}
|
||||
|
@ -1114,7 +1109,6 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
|
|||
}
|
||||
|
||||
/** Clears the cache. Used in tests. */
|
||||
@VisibleForTesting
|
||||
public void clearCache() {
|
||||
this.map.clear();
|
||||
this.elements.set(0);
|
||||
|
@ -1124,7 +1118,6 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
|
|||
* Used in testing. May be very inefficient.
|
||||
* @return the set of cached file names
|
||||
*/
|
||||
@VisibleForTesting
|
||||
SortedSet<String> getCachedFileNamesForTest() {
|
||||
SortedSet<String> fileNames = new TreeSet<String>();
|
||||
for (BlockCacheKey cacheKey : map.keySet()) {
|
||||
|
@ -1133,7 +1126,6 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
|
|||
return fileNames;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
Map<BlockType, Integer> getBlockTypeCountsForTest() {
|
||||
Map<BlockType, Integer> counts =
|
||||
new EnumMap<BlockType, Integer>(BlockType.class);
|
||||
|
@ -1145,7 +1137,6 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
|
|||
return counts;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public Map<DataBlockEncoding, Integer> getEncodingCountsForTest() {
|
||||
Map<DataBlockEncoding, Integer> counts =
|
||||
new EnumMap<DataBlockEncoding, Integer>(DataBlockEncoding.class);
|
||||
|
@ -1163,7 +1154,6 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
|
|||
victimHandler = handler;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
Map<BlockCacheKey, LruCachedBlock> getMapForTests() {
|
||||
return map;
|
||||
}
|
||||
|
|
|
@ -20,6 +20,8 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.io.hfile.bucket;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.FileNotFoundException;
|
||||
|
@ -50,8 +52,6 @@ import java.util.concurrent.atomic.AtomicLong;
|
|||
import java.util.concurrent.locks.Lock;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -76,9 +76,6 @@ import org.apache.hadoop.hbase.util.HasThread;
|
|||
import org.apache.hadoop.hbase.util.IdReadWriteLock;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
|
||||
/**
|
||||
* BucketCache uses {@link BucketAllocator} to allocate/free blocks, and uses
|
||||
* BucketCache#ramCache and BucketCache#backingMap in order to
|
||||
|
@ -110,7 +107,6 @@ public class BucketCache implements BlockCache, HeapSize {
|
|||
static final String MIN_FACTOR_CONFIG_NAME = "hbase.bucketcache.minfactor";
|
||||
|
||||
/** Priority buckets */
|
||||
@VisibleForTesting
|
||||
static final float DEFAULT_SINGLE_FACTOR = 0.25f;
|
||||
static final float DEFAULT_MULTI_FACTOR = 0.50f;
|
||||
static final float DEFAULT_MEMORY_FACTOR = 0.25f;
|
||||
|
@ -132,10 +128,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 ConcurrentMap<BlockCacheKey, RAMQueueEntry> ramCache;
|
||||
// In this map, store the block's meta data like offset, length
|
||||
@VisibleForTesting
|
||||
transient ConcurrentMap<BlockCacheKey, BucketEntry> backingMap;
|
||||
|
||||
/**
|
||||
|
@ -152,10 +146,8 @@ 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<BlockingQueue<RAMQueueEntry>>();
|
||||
@VisibleForTesting
|
||||
transient final WriterThread[] writerThreads;
|
||||
|
||||
/** Volatile boolean to track if free space is in process or not */
|
||||
|
@ -198,7 +190,6 @@ public class BucketCache implements BlockCache, HeapSize {
|
|||
* A ReentrantReadWriteLock to lock on a particular block identified by offset.
|
||||
* The purpose of this is to avoid freeing the block which is being read.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
transient final IdReadWriteLock offsetLock = new IdReadWriteLock();
|
||||
|
||||
private final NavigableSet<BlockCacheKey> blocksByHFile =
|
||||
|
@ -350,14 +341,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;
|
||||
}
|
||||
|
@ -550,7 +539,6 @@ public class BucketCache implements BlockCache, HeapSize {
|
|||
return null;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
void blockEvicted(BlockCacheKey cacheKey, BucketEntry bucketEntry, boolean decrementBlockNumber) {
|
||||
bucketAllocator.freeBlock(bucketEntry.offset());
|
||||
realCacheSize.addAndGet(-1 * bucketEntry.getLength());
|
||||
|
@ -646,7 +634,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);
|
||||
}
|
||||
|
@ -840,7 +827,6 @@ public class BucketCache implements BlockCache, HeapSize {
|
|||
}
|
||||
|
||||
// This handles flushing the RAM cache to IOEngine.
|
||||
@VisibleForTesting
|
||||
class WriterThread extends HasThread {
|
||||
private final BlockingQueue<RAMQueueEntry> inputQueue;
|
||||
private volatile boolean writerEnabled = true;
|
||||
|
@ -851,7 +837,6 @@ public class BucketCache implements BlockCache, HeapSize {
|
|||
}
|
||||
|
||||
// Used for test
|
||||
@VisibleForTesting
|
||||
void disableWriter() {
|
||||
this.writerEnabled = false;
|
||||
}
|
||||
|
@ -912,7 +897,6 @@ public class BucketCache implements BlockCache, HeapSize {
|
|||
* interference expected.
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
@VisibleForTesting
|
||||
void doDrain(final List<RAMQueueEntry> entries) throws InterruptedException {
|
||||
if (entries.isEmpty()) {
|
||||
return;
|
||||
|
@ -1021,7 +1005,6 @@ public class BucketCache implements BlockCache, HeapSize {
|
|||
* @param q The queue to take from.
|
||||
* @return receptical laden with elements taken from the queue or empty if none found.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
static List<RAMQueueEntry> getRAMQueueEntries(final BlockingQueue<RAMQueueEntry> q,
|
||||
final List<RAMQueueEntry> receptical)
|
||||
throws InterruptedException {
|
||||
|
@ -1448,7 +1431,6 @@ public class BucketCache implements BlockCache, HeapSize {
|
|||
/**
|
||||
* Block Entry stored in the memory with key,data and so on
|
||||
*/
|
||||
@VisibleForTesting
|
||||
static class RAMQueueEntry {
|
||||
private BlockCacheKey key;
|
||||
private Cacheable data;
|
||||
|
@ -1644,7 +1626,6 @@ public class BucketCache implements BlockCache, HeapSize {
|
|||
return memoryFactor;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public UniqueIndexMap<Integer> getDeserialiserMap() {
|
||||
return deserialiserMap;
|
||||
}
|
||||
|
|
|
@ -30,7 +30,6 @@ import java.security.NoSuchAlgorithmException;
|
|||
import java.util.Arrays;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
@ -138,7 +137,6 @@ public class FileIOEngine implements PersistentIOEngine {
|
|||
return 0;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
void closeFileChannels() {
|
||||
for (FileChannel fileChannel: fileChannels) {
|
||||
try {
|
||||
|
@ -262,12 +260,10 @@ public class FileIOEngine implements PersistentIOEngine {
|
|||
return fileNum;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
FileChannel[] getFileChannels() {
|
||||
return fileChannels;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
void refreshFileConnection(int accessFileNum, IOException ioe) throws IOException {
|
||||
ReentrantLock channelLock = channelLocks[accessFileNum];
|
||||
channelLock.lock();
|
||||
|
|
|
@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.ipc;
|
|||
|
||||
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
import com.google.protobuf.BlockingService;
|
||||
import com.google.protobuf.CodedInputStream;
|
||||
|
@ -2612,7 +2611,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
|
|||
* @param strParam stringifiedParam to be truncated
|
||||
* @return truncated trace log string
|
||||
*/
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
String truncateTraceLog(String strParam) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
int traceLogMaxLength = getConf().getInt(TRACE_LOG_MAX_LENGTH, DEFAULT_TRACE_LOG_MAX_LENGTH);
|
||||
|
|
|
@ -19,9 +19,12 @@
|
|||
|
||||
package org.apache.hadoop.hbase.ipc;
|
||||
|
||||
import com.google.protobuf.BlockingService;
|
||||
import com.google.protobuf.Descriptors.MethodDescriptor;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
|
@ -33,12 +36,6 @@ import org.apache.hadoop.hbase.regionserver.RSRpcServices;
|
|||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.security.authorize.PolicyProvider;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.protobuf.BlockingService;
|
||||
import com.google.protobuf.Descriptors.MethodDescriptor;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
|
||||
@InterfaceStability.Evolving
|
||||
public interface RpcServerInterface {
|
||||
|
@ -84,7 +81,7 @@ public interface RpcServerInterface {
|
|||
* for correctness.
|
||||
* @param pp PolicyProvider
|
||||
*/
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
@Deprecated
|
||||
void refreshAuthManager(PolicyProvider pp);
|
||||
|
||||
|
@ -93,7 +90,7 @@ public interface RpcServerInterface {
|
|||
* @param conf configuration for refresh
|
||||
* @param pp
|
||||
*/
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
void refreshAuthManager(Configuration conf, PolicyProvider pp);
|
||||
|
||||
RpcScheduler getScheduler();
|
||||
|
|
|
@ -23,8 +23,6 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.util.ProgramDriver;
|
||||
|
||||
import 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;
|
||||
}
|
||||
|
|
|
@ -40,8 +40,6 @@ import org.apache.hadoop.mapreduce.RecordWriter;
|
|||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* Writes HFiles. Passed KeyValues must arrive in order.
|
||||
* Writes current time as the sequence id for the file. Sets the major compacted
|
||||
|
@ -99,7 +97,7 @@ public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable,
|
|||
* @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) {
|
||||
return HFileOutputFormat2.createFamilyCompressionMap(conf);
|
||||
|
@ -112,7 +110,7 @@ public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable,
|
|||
* @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) {
|
||||
return HFileOutputFormat2.createFamilyBloomTypeMap(conf);
|
||||
}
|
||||
|
@ -124,7 +122,7 @@ public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable,
|
|||
* @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) {
|
||||
return HFileOutputFormat2.createFamilyBlockSizeMap(conf);
|
||||
}
|
||||
|
@ -137,7 +135,7 @@ public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable,
|
|||
* @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) {
|
||||
return HFileOutputFormat2.createFamilyDataBlockEncodingMap(conf);
|
||||
|
@ -165,7 +163,7 @@ public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable,
|
|||
* @throws IOException
|
||||
* on failure to read column family descriptors
|
||||
*/
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
static void configureBlockSize(Table table, Configuration conf) throws IOException {
|
||||
HFileOutputFormat2.configureBlockSize(table.getTableDescriptor(), conf);
|
||||
}
|
||||
|
@ -179,7 +177,7 @@ public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable,
|
|||
* @throws IOException
|
||||
* on failure to read column family descriptors
|
||||
*/
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
static void configureBloomType(Table table, Configuration conf) throws IOException {
|
||||
HFileOutputFormat2.configureBloomType(table.getTableDescriptor(), conf);
|
||||
}
|
||||
|
@ -193,7 +191,7 @@ public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable,
|
|||
* @throws IOException
|
||||
* on failure to read column family descriptors
|
||||
*/
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
static void configureDataBlockEncoding(Table table,
|
||||
Configuration conf) throws IOException {
|
||||
HTableDescriptor tableDescriptor = table.getTableDescriptor();
|
||||
|
|
|
@ -17,8 +17,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.mapreduce;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.UnsupportedEncodingException;
|
||||
import java.net.InetSocketAddress;
|
||||
|
@ -583,7 +581,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,
|
||||
|
@ -605,7 +603,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);
|
||||
|
@ -625,7 +623,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);
|
||||
|
@ -646,7 +644,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,
|
||||
|
@ -720,7 +718,7 @@ public class HFileOutputFormat2
|
|||
*/
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(
|
||||
value="RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE")
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
static void configureCompression(Configuration conf, HTableDescriptor tableDescriptor)
|
||||
throws UnsupportedEncodingException {
|
||||
StringBuilder compressionConfigValue = new StringBuilder();
|
||||
|
@ -753,7 +751,7 @@ public class HFileOutputFormat2
|
|||
* @throws IOException
|
||||
* on failure to read column family descriptors
|
||||
*/
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
static void configureBlockSize(HTableDescriptor tableDescriptor, Configuration conf)
|
||||
throws UnsupportedEncodingException {
|
||||
StringBuilder blockSizeConfigValue = new StringBuilder();
|
||||
|
@ -786,7 +784,7 @@ public class HFileOutputFormat2
|
|||
* @throws IOException
|
||||
* on failure to read column family descriptors
|
||||
*/
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
static void configureBloomType(HTableDescriptor tableDescriptor, Configuration conf)
|
||||
throws UnsupportedEncodingException {
|
||||
if (tableDescriptor == null) {
|
||||
|
@ -821,7 +819,7 @@ public class HFileOutputFormat2
|
|||
* @throws IOException
|
||||
* on failure to read column family descriptors
|
||||
*/
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
static void configureDataBlockEncoding(HTableDescriptor tableDescriptor,
|
||||
Configuration conf) throws UnsupportedEncodingException {
|
||||
if (tableDescriptor == null) {
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.mapreduce;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import static java.lang.String.format;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
|
|
|
@ -40,8 +40,6 @@ import org.apache.hadoop.mapreduce.InputSplit;
|
|||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* Iterate over an HBase table data, return (ImmutableBytesWritable, Result)
|
||||
* pairs.
|
||||
|
@ -55,8 +53,9 @@ public class TableRecordReaderImpl {
|
|||
private static final Log LOG = LogFactory.getLog(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 = "HBase Counters";
|
||||
|
||||
private ResultScanner scanner = null;
|
||||
private Scan scan = null;
|
||||
private Scan currentScan = null;
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
|
||||
package org.apache.hadoop.hbase.mapreduce;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
|
@ -131,7 +130,7 @@ public class TableSnapshotInputFormat extends InputFormat<ImmutableBytesWritable
|
|||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
static class TableSnapshotRegionRecordReader extends
|
||||
RecordReader<ImmutableBytesWritable, Result> {
|
||||
private TableSnapshotInputFormatImpl.RecordReader delegate =
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.master;
|
||||
|
||||
import com.google.common.collect.LinkedHashMultimap;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
|
@ -43,7 +44,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.locks.Lock;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -118,9 +118,6 @@ import org.apache.zookeeper.KeeperException.NoNodeException;
|
|||
import org.apache.zookeeper.KeeperException.NodeExistsException;
|
||||
import org.apache.zookeeper.data.Stat;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.collect.LinkedHashMultimap;
|
||||
|
||||
/**
|
||||
* Manages and performs region assignment.
|
||||
* <p>
|
||||
|
@ -403,7 +400,6 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
/**
|
||||
* Used in some tests to mock up region state in meta
|
||||
*/
|
||||
@VisibleForTesting
|
||||
RegionStateStore getRegionStateStore() {
|
||||
return regionStateStore;
|
||||
}
|
||||
|
@ -1708,7 +1704,6 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
/**
|
||||
* Use care with forceNewPlan. It could cause double assignment.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public void assign(HRegionInfo region,
|
||||
boolean setOfflineInZK, boolean forceNewPlan) {
|
||||
if (isDisabledorDisablingRegionInRIT(region)) {
|
||||
|
@ -4798,7 +4793,6 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
/*
|
||||
* This is only used for unit-testing split failures.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public static void setTestSkipSplitHandling(boolean skipSplitHandling) {
|
||||
TEST_SKIP_SPLIT_HANDLING = skipSplitHandling;
|
||||
}
|
||||
|
@ -4806,7 +4800,6 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
/*
|
||||
* This is only used for unit-testing merge failures.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public static void setTestSkipMergeHandling(boolean skipMergeHandling) {
|
||||
TEST_SKIP_MERGE_HANDLING = skipMergeHandling;
|
||||
}
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
|
||||
package org.apache.hadoop.hbase.master;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
@ -148,7 +147,6 @@ public class CachedClusterId {
|
|||
return getClusterId();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public int getCacheStats() {
|
||||
return cacheMisses.get();
|
||||
}
|
||||
|
|
|
@ -17,6 +17,8 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.master;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.protobuf.Descriptors;
|
||||
import com.google.protobuf.Service;
|
||||
import java.io.IOException;
|
||||
|
@ -40,12 +42,10 @@ import java.util.concurrent.CountDownLatch;
|
|||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import javax.servlet.ServletException;
|
||||
import javax.servlet.http.HttpServlet;
|
||||
import javax.servlet.http.HttpServletRequest;
|
||||
import javax.servlet.http.HttpServletResponse;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -187,10 +187,6 @@ import org.mortbay.jetty.nio.SelectChannelConnector;
|
|||
import org.mortbay.jetty.servlet.Context;
|
||||
import org.mortbay.jetty.servlet.ServletHolder;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
|
||||
/**
|
||||
* run the cluster. All others park themselves in their constructor until
|
||||
* master or cluster shutdown or until the active master loses its lease in
|
||||
|
@ -610,7 +606,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
public MasterRpcServices getMasterRpcServices() {
|
||||
return (MasterRpcServices)rpcServices;
|
||||
}
|
||||
|
@ -1746,7 +1742,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
this.catalogJanitorChore, region_a, region_b, forcible, user));
|
||||
}
|
||||
|
||||
@VisibleForTesting // Public so can be accessed by tests.
|
||||
@InterfaceAudience.Private // Public so can be accessed by tests.
|
||||
public void move(final byte[] encodedRegionName,
|
||||
byte[] destServerName) throws HBaseIOException {
|
||||
RegionState regionState = assignmentManager.getRegionStates().
|
||||
|
@ -2454,7 +2450,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
* is found, but not currently deployed, the second element of the pair
|
||||
* may be null.
|
||||
*/
|
||||
@VisibleForTesting // Used by TestMaster.
|
||||
@InterfaceAudience.Private // Used by TestMaster.
|
||||
Pair<HRegionInfo, ServerName> getTableRegionForRow(
|
||||
final TableName tableName, final byte [] rowKey)
|
||||
throws IOException {
|
||||
|
@ -2738,7 +2734,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
stop("Stopped by " + Thread.currentThread().getName());
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
protected void checkServiceStarted() throws ServerNotRunningYetException {
|
||||
if (!serviceStarted) {
|
||||
throw new ServerNotRunningYetException("Server is not running yet");
|
||||
|
@ -2798,7 +2794,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
return maintenanceModeTracker.isInMaintenanceMode();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
public void setInitialized(boolean isInitialized) {
|
||||
procedureExecutor.getEnvironment().setEventReady(initialized, isInitialized);
|
||||
}
|
||||
|
@ -2817,7 +2813,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
return serverCrashProcessingEnabled.isReady();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
public void setServerCrashProcessingEnabled(final boolean b) {
|
||||
procedureExecutor.getEnvironment().setEventReady(serverCrashProcessingEnabled, b);
|
||||
}
|
||||
|
|
|
@ -58,8 +58,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
|||
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* This class abstracts a bunch of operations the HMaster needs to interact with
|
||||
* the underlying file system, including splitting log files, checking file
|
||||
|
@ -147,7 +145,6 @@ public class MasterFileSystem {
|
|||
this.distributedLogReplay = this.splitLogManager.isLogReplaying();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
SplitLogManager getSplitLogManager() {
|
||||
return this.splitLogManager;
|
||||
}
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.master;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
|
@ -30,9 +31,6 @@ import java.util.Comparator;
|
|||
import java.util.SortedSet;
|
||||
import java.util.TreeMap;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -611,7 +609,6 @@ public class RegionStates {
|
|||
/**
|
||||
* Used in some unit tests
|
||||
*/
|
||||
@VisibleForTesting
|
||||
synchronized boolean existsInServerHoldings(final ServerName serverName,
|
||||
final HRegionInfo hri) {
|
||||
Set<HRegionInfo> oldRegions = serverHoldings.get(serverName);
|
||||
|
@ -988,7 +985,6 @@ public class RegionStates {
|
|||
/**
|
||||
* Remove a region from all state maps.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public synchronized void deleteRegion(final HRegionInfo hri) {
|
||||
String encodedName = hri.getEncodedName();
|
||||
regionsInTransition.remove(encodedName);
|
||||
|
@ -1006,7 +1002,6 @@ public class RegionStates {
|
|||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public boolean isRegionInRegionStates(final HRegionInfo hri) {
|
||||
return (getRegionState(hri) != null || isRegionOnline(hri)) || isRegionInTransition(hri)
|
||||
|| isRegionInState(hri, State.OFFLINE, State.CLOSED);
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.master;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import java.io.IOException;
|
||||
|
@ -472,7 +471,6 @@ public class ServerManager {
|
|||
* @param sl
|
||||
* @return Server load from the removed server, if any.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
void recordNewServerWithLock(final ServerName serverName, final ServerLoad sl) {
|
||||
LOG.info("Registering server=" + serverName);
|
||||
this.onlineServers.put(serverName, sl);
|
||||
|
@ -655,7 +653,6 @@ public class ServerManager {
|
|||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public void moveFromOnlineToDeadServers(final ServerName sn) {
|
||||
synchronized (onlineServers) {
|
||||
if (!this.onlineServers.containsKey(sn)) {
|
||||
|
@ -1312,7 +1309,6 @@ public class ServerManager {
|
|||
flushedSequenceIdByRegion.remove(encodedName);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public boolean isRegionInServerManagerStates(final HRegionInfo hri) {
|
||||
final byte[] encodedName = hri.getEncodedNameAsBytes();
|
||||
return (storeFlushedSequenceIdsByRegion.containsKey(encodedName)
|
||||
|
|
|
@ -65,8 +65,6 @@ import org.apache.hadoop.hbase.util.Pair;
|
|||
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
|
||||
import org.apache.hadoop.hbase.wal.WALFactory;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* Distributes the task of log splitting to the available region servers.
|
||||
* Coordination happens via coordination engine. For every log file that has to be split a
|
||||
|
@ -176,7 +174,6 @@ public class SplitLogManager {
|
|||
* {@link org.apache.hadoop.hbase.wal.WALSplitter#split(Path, Path, Path, FileSystem,
|
||||
* Configuration, WALFactory)} for tests.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public static FileStatus[] getFileList(final Configuration conf, final List<Path> logDirs,
|
||||
final PathFilter filter)
|
||||
throws IOException {
|
||||
|
@ -378,7 +375,6 @@ public class SplitLogManager {
|
|||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
ConcurrentMap<String, Task> getTasks() {
|
||||
return tasks;
|
||||
}
|
||||
|
|
|
@ -17,6 +17,10 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.master.balancer;
|
||||
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.collect.ArrayListMultimap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
|
@ -33,7 +37,6 @@ import java.util.NavigableMap;
|
|||
import java.util.Random;
|
||||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.commons.lang.NotImplementedException;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -45,6 +48,7 @@ import org.apache.hadoop.hbase.HDFSBlocksDistribution;
|
|||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.RegionLoad;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||
import org.apache.hadoop.hbase.master.LoadBalancer;
|
||||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
|
@ -54,12 +58,6 @@ import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.Action.T
|
|||
import org.apache.hadoop.hbase.namequeues.NamedQueueRecorder;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.collect.ArrayListMultimap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
|
||||
/**
|
||||
* The base class for load balancers. It provides the the functions used to by
|
||||
* {@link org.apache.hadoop.hbase.master.AssignmentManager} to assign regions
|
||||
|
@ -1065,12 +1063,12 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
|
|||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
protected void setNumRegions(int numRegions) {
|
||||
this.numRegions = numRegions;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
protected void setNumMovedRegions(int numMovedRegions) {
|
||||
this.numMovedRegions = numMovedRegions;
|
||||
}
|
||||
|
|
|
@ -17,7 +17,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.master.balancer;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import java.util.ArrayDeque;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
|
@ -386,7 +385,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
|
|||
return balanceCluster(clusterState);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
Cluster.Action nextAction(Cluster cluster) {
|
||||
return candidateGenerators[(RANDOM.nextInt(candidateGenerators.length))]
|
||||
.generate(cluster);
|
||||
|
|
|
@ -17,6 +17,10 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.master.cleaner;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
|
@ -25,7 +29,6 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -38,12 +41,6 @@ import org.apache.hadoop.hbase.Stoppable;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
/**
|
||||
* Abstract Cleaner that uses a chain of delegates to clean a directory of files
|
||||
* @param <T> Cleaner delegate class that is dynamically loaded from configuration
|
||||
|
@ -310,7 +307,6 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
|
|||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
int getChorePoolSize() {
|
||||
return pool.getSize();
|
||||
}
|
||||
|
|
|
@ -17,7 +17,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.master.cleaner;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
@ -80,12 +79,10 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
|
|||
|
||||
public static final String HFILE_DELETE_THREAD_TIMEOUT_MSEC =
|
||||
"hbase.regionserver.hfilecleaner.thread.timeout.msec";
|
||||
@VisibleForTesting
|
||||
static final long DEFAULT_HFILE_DELETE_THREAD_TIMEOUT_MSEC = 60 * 1000L;
|
||||
|
||||
public static final String HFILE_DELETE_THREAD_CHECK_INTERVAL_MSEC =
|
||||
"hbase.regionserver.hfilecleaner.thread.check.interval.msec";
|
||||
@VisibleForTesting
|
||||
static final long DEFAULT_HFILE_DELETE_THREAD_CHECK_INTERVAL_MSEC = 1000L;
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(HFileCleaner.class);
|
||||
|
@ -387,42 +384,34 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> impleme
|
|||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public List<Thread> getCleanerThreads() {
|
||||
return threads;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public long getNumOfDeletedLargeFiles() {
|
||||
return deletedLargeFiles.get();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public long getNumOfDeletedSmallFiles() {
|
||||
return deletedSmallFiles.get();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public long getLargeQueueInitSize() {
|
||||
return largeQueueInitSize;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public long getSmallQueueInitSize() {
|
||||
return smallQueueInitSize;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public long getThrottlePoint() {
|
||||
return throttlePoint;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
long getCleanerThreadTimeoutMsec() {
|
||||
return cleanerThreadTimeoutMsec;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
long getCleanerThreadCheckIntervalMsec() {
|
||||
return cleanerThreadCheckIntervalMsec;
|
||||
}
|
||||
|
|
|
@ -19,8 +19,6 @@ package org.apache.hadoop.hbase.master.cleaner;
|
|||
|
||||
import static org.apache.hadoop.hbase.HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.LinkedList;
|
||||
|
@ -54,15 +52,12 @@ public class LogCleaner extends CleanerChore<BaseLogCleanerDelegate>
|
|||
|
||||
public static final String OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC =
|
||||
"hbase.oldwals.cleaner.thread.timeout.msec";
|
||||
@VisibleForTesting
|
||||
static final long DEFAULT_OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC = 60 * 1000L;
|
||||
|
||||
public static final String OLD_WALS_CLEANER_THREAD_CHECK_INTERVAL_MSEC =
|
||||
"hbase.oldwals.cleaner.thread.check.interval.msec";
|
||||
@VisibleForTesting
|
||||
static final long DEFAULT_OLD_WALS_CLEANER_THREAD_CHECK_INTERVAL_MSEC = 500L;
|
||||
|
||||
|
||||
private final LinkedBlockingQueue<CleanerContext> pendingDelete;
|
||||
private List<Thread> oldWALsCleaner;
|
||||
private long cleanerThreadTimeoutMsec;
|
||||
|
@ -135,17 +130,14 @@ public class LogCleaner extends CleanerChore<BaseLogCleanerDelegate>
|
|||
interruptOldWALsCleaner();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
int getSizeOfCleaners() {
|
||||
return oldWALsCleaner.size();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
long getCleanerThreadTimeoutMsec() {
|
||||
return cleanerThreadTimeoutMsec;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
long getCleanerThreadCheckIntervalMsec() {
|
||||
return cleanerThreadCheckIntervalMsec;
|
||||
}
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.master.cleaner;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
@ -40,7 +39,6 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
|||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.zookeeper.data.Stat;
|
||||
|
||||
|
||||
/**
|
||||
* A cleaner that cleans replication locks on zk which is locked by dead region servers
|
||||
*/
|
||||
|
@ -55,7 +53,6 @@ public class ReplicationZKLockCleanerChore extends ScheduledChore {
|
|||
// Wait some times before delete lock to prevent a session expired RS not dead fully.
|
||||
private static final long DEFAULT_TTL = 60 * 10 * 1000;//10 min
|
||||
|
||||
@VisibleForTesting
|
||||
public static final String TTL_CONFIG_KEY = "hbase.replication.zk.deadrs.lock.ttl";
|
||||
|
||||
public ReplicationZKLockCleanerChore(Stoppable stopper, Abortable abortable, int period,
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
|
||||
package org.apache.hadoop.hbase.master.procedure;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
|
@ -287,7 +286,6 @@ public class TruncateTableProcedure
|
|||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
HRegionInfo getFirstRegionInfo() {
|
||||
if (regions == null || regions.isEmpty()) {
|
||||
return null;
|
||||
|
|
|
@ -17,8 +17,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.master.snapshot;
|
||||
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
|
@ -185,7 +183,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
|
|||
* @param coordinator procedure coordinator instance. exposed for testing.
|
||||
* @param pool HBase ExecutorServcie instance, exposed for testing.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
SnapshotManager(final MasterServices master, ProcedureCoordinator coordinator,
|
||||
ExecutorService pool, int sentinelCleanInterval)
|
||||
throws IOException, UnsupportedOperationException {
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.procedure;
|
||||
|
||||
import com.google.common.collect.MapMaker;
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.HashSet;
|
||||
|
@ -28,16 +29,13 @@ import java.util.concurrent.RejectedExecutionException;
|
|||
import java.util.concurrent.SynchronousQueue;
|
||||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.DaemonThreadFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.errorhandling.ForeignException;
|
||||
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
|
||||
|
||||
import com.google.common.collect.MapMaker;
|
||||
|
||||
/**
|
||||
* This is the master side of a distributed complex procedure execution.
|
||||
* <p>
|
||||
|
|
|
@ -13,7 +13,6 @@ package org.apache.hadoop.hbase.quotas;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* This limiter will refill resources at every TimeUnit/resources interval. For example: For a
|
||||
|
@ -62,13 +61,11 @@ public class AverageIntervalRateLimiter extends RateLimiter {
|
|||
}
|
||||
|
||||
// This method is for strictly testing purpose only
|
||||
@VisibleForTesting
|
||||
@Override
|
||||
public void setNextRefillTime(long nextRefillTime) {
|
||||
this.nextRefillTime = nextRefillTime;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@Override
|
||||
public long getNextRefillTime() {
|
||||
return this.nextRefillTime;
|
||||
|
|
|
@ -13,7 +13,6 @@ package org.apache.hadoop.hbase.quotas;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* With this limiter resources will be refilled only after a fixed interval of time.
|
||||
|
@ -44,13 +43,11 @@ public class FixedIntervalRateLimiter extends RateLimiter {
|
|||
}
|
||||
|
||||
// This method is for strictly testing purpose only
|
||||
@VisibleForTesting
|
||||
@Override
|
||||
public void setNextRefillTime(long nextRefillTime) {
|
||||
this.nextRefillTime = nextRefillTime;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@Override
|
||||
public long getNextRefillTime() {
|
||||
return this.nextRefillTime;
|
||||
|
|
|
@ -31,8 +31,6 @@ import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
|||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* Cache that keeps track of the quota settings for the users and tables that are interacting with
|
||||
* it. To avoid blocking the operations if the requested quota is not in cache an "empty quota" will
|
||||
|
@ -151,27 +149,22 @@ public class QuotaCache implements Stoppable {
|
|||
return quotaInfo;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
void triggerCacheRefresh() {
|
||||
refreshChore.triggerNow();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
long getLastUpdate() {
|
||||
return refreshChore.lastUpdate;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
Map<String, QuotaState> getNamespaceQuotaCache() {
|
||||
return namespaceQuotaCache;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
Map<TableName, QuotaState> getTableQuotaCache() {
|
||||
return tableQuotaCache;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
Map<String, UserQuotaState> getUserQuotaCache() {
|
||||
return userQuotaCache;
|
||||
}
|
||||
|
|
|
@ -16,8 +16,6 @@ import java.util.concurrent.TimeUnit;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* Simple rate limiter.
|
||||
*
|
||||
|
@ -222,9 +220,8 @@ public abstract class RateLimiter {
|
|||
}
|
||||
|
||||
// These two method are for strictly testing purpose only
|
||||
@VisibleForTesting
|
||||
|
||||
public abstract void setNextRefillTime(long nextRefillTime);
|
||||
|
||||
@VisibleForTesting
|
||||
public abstract long getNextRefillTime();
|
||||
}
|
||||
|
|
|
@ -28,8 +28,6 @@ import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
|||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* Region Server Quota Manager. It is responsible to provide access to the quota information of each
|
||||
* user/table. The direct user of this class is the RegionServer that will get and check the
|
||||
|
@ -77,7 +75,6 @@ public class RegionServerQuotaManager {
|
|||
return quotaCache != null;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
QuotaCache getQuotaCache() {
|
||||
return quotaCache;
|
||||
}
|
||||
|
|
|
@ -17,10 +17,11 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.TextFormat;
|
||||
import java.lang.reflect.Method;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -39,10 +40,6 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.TextFormat;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
|
||||
/**
|
||||
|
@ -279,7 +276,6 @@ public class AnnotationReadingPriorityFunction implements PriorityFunction {
|
|||
return 0;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
void setRegionServer(final HRegionServer hrs) {
|
||||
this.rpcServices = hrs.getRSRpcServices();
|
||||
}
|
||||
|
|
|
@ -18,8 +18,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.Comparator;
|
||||
import java.util.Iterator;
|
||||
|
@ -188,7 +186,6 @@ public class CellSkipListSet implements NavigableSet<Cell> {
|
|||
throw new UnsupportedOperationException("Not implemented");
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
int sizeForTests() {
|
||||
return this.delegatee.size();
|
||||
}
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import java.io.IOException;
|
||||
import java.io.PrintWriter;
|
||||
import java.io.StringWriter;
|
||||
|
@ -32,7 +33,6 @@ import java.util.concurrent.ThreadFactory;
|
|||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -50,9 +50,6 @@ import org.apache.hadoop.hbase.util.Pair;
|
|||
import org.apache.hadoop.hbase.util.StealJobQueue;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* Compact region on request and then run split if appropriate
|
||||
*/
|
||||
|
@ -775,12 +772,10 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
|
|||
// No children to register
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public ThroughputController getCompactionThroughputController() {
|
||||
return compactionThroughputController;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
/**
|
||||
* Shutdown the long compaction thread pool.
|
||||
* Should only be used in unit test to prevent long compaction thread pool from stealing job
|
||||
|
|
|
@ -27,8 +27,6 @@ import org.apache.hadoop.hbase.Stoppable;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.executor.EventType;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* A chore service that periodically cleans up the compacted files when there are no active readers
|
||||
* using those compacted files and also helps in clearing the block cache with these compacted
|
||||
|
@ -39,7 +37,6 @@ public class CompactedHFilesDischarger extends ScheduledChore {
|
|||
private static final Log LOG = LogFactory.getLog(CompactedHFilesDischarger.class);
|
||||
private RegionServerServices regionServerServices;
|
||||
// Default is to use executor
|
||||
@VisibleForTesting
|
||||
private boolean useExecutor = true;
|
||||
|
||||
/**
|
||||
|
@ -60,7 +57,6 @@ public class CompactedHFilesDischarger extends ScheduledChore {
|
|||
* @param regionServerServices the region server that starts this chore
|
||||
* @param useExecutor true if to use the region server's executor service, false otherwise
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public CompactedHFilesDischarger(final int period, final Stoppable stopper,
|
||||
final RegionServerServices regionServerServices, boolean useExecutor) {
|
||||
// Need to add the config classes
|
||||
|
|
|
@ -17,8 +17,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -91,7 +89,7 @@ public class ConstantSizeRegionSplitPolicy extends RegionSplitPolicy {
|
|||
return desiredMaxFileSize;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@InterfaceAudience.Private
|
||||
public boolean positiveJitterRate() {
|
||||
return this.jitterRate > 0;
|
||||
}
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue