HBASE-11677 Making Logger instance modifiers consistent.
* IA.Public accessible logger instances deprecated * logger instances modified by tests left in place * all others made private static final Signed-off-by: Sean Busbey <busbey@apache.org>
This commit is contained in:
parent
84dd84fc58
commit
94a00bea07
|
@ -96,7 +96,7 @@ import com.google.common.annotations.VisibleForTesting;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
class AsyncProcess {
|
||||
protected static final Log LOG = LogFactory.getLog(AsyncProcess.class);
|
||||
private static final Log LOG = LogFactory.getLog(AsyncProcess.class);
|
||||
protected static final AtomicLong COUNTER = new AtomicLong();
|
||||
|
||||
public static final String PRIMARY_CALL_TIMEOUT_KEY = "hbase.client.primaryCallTimeout.multiget";
|
||||
|
|
|
@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
final class ClientIdGenerator {
|
||||
static final Log LOG = LogFactory.getLog(ClientIdGenerator.class);
|
||||
private static final Log LOG = LogFactory.getLog(ClientIdGenerator.class);
|
||||
|
||||
private ClientIdGenerator() {}
|
||||
|
||||
|
|
|
@ -54,7 +54,7 @@ import com.google.common.annotations.VisibleForTesting;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ClientScanner extends AbstractClientScanner {
|
||||
private final Log LOG = LogFactory.getLog(this.getClass());
|
||||
private static final Log LOG = LogFactory.getLog(ClientScanner.class);
|
||||
// A byte array in which all elements are the max byte, and it is used to
|
||||
// construct closest front row
|
||||
static byte[] MAX_BYTE_ARRAY = Bytes.createMaxByteArray(9);
|
||||
|
|
|
@ -51,7 +51,7 @@ import com.google.protobuf.ServiceException;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ClientSmallScanner extends ClientScanner {
|
||||
private final Log LOG = LogFactory.getLog(this.getClass());
|
||||
private static final Log LOG = LogFactory.getLog(ClientSmallScanner.class);
|
||||
private ScannerCallableWithReplicas smallScanCallable = null;
|
||||
private SmallScannerCallableFactory callableFactory;
|
||||
|
||||
|
|
|
@ -98,7 +98,7 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
@InterfaceAudience.Private
|
||||
class ConnectionImplementation implements ClusterConnection, Closeable {
|
||||
public static final String RETRIES_BY_SERVER_KEY = "hbase.client.retries.by.server";
|
||||
static final Log LOG = LogFactory.getLog(ConnectionImplementation.class);
|
||||
private static final Log LOG = LogFactory.getLog(ConnectionImplementation.class);
|
||||
private static final String CLIENT_NONCES_ENABLED_KEY = "hbase.client.nonces.enabled";
|
||||
|
||||
private final long pause;
|
||||
|
|
|
@ -73,7 +73,7 @@ import org.apache.hadoop.ipc.RemoteException;
|
|||
@InterfaceAudience.Private
|
||||
class PreemptiveFastFailInterceptor extends RetryingCallerInterceptor {
|
||||
|
||||
public static final Log LOG = LogFactory
|
||||
private static final Log LOG = LogFactory
|
||||
.getLog(PreemptiveFastFailInterceptor.class);
|
||||
|
||||
// amount of time to wait before we consider a server to be in fast fail
|
||||
|
|
|
@ -47,7 +47,7 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
@InterfaceAudience.Private
|
||||
public abstract class RegionServerCallable<T> implements RetryingCallable<T> {
|
||||
// Public because used outside of this package over in ipc.
|
||||
static final Log LOG = LogFactory.getLog(RegionServerCallable.class);
|
||||
private static final Log LOG = LogFactory.getLog(RegionServerCallable.class);
|
||||
protected final Connection connection;
|
||||
protected final TableName tableName;
|
||||
protected final byte[] row;
|
||||
|
|
|
@ -49,6 +49,7 @@ import com.google.protobuf.ServiceException;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class RpcRetryingCallerImpl<T> implements RpcRetryingCaller<T> {
|
||||
// LOG is being used in TestMultiRowRangeFilter, hence leaving it public
|
||||
public static final Log LOG = LogFactory.getLog(RpcRetryingCallerImpl.class);
|
||||
/**
|
||||
* When we started making calls.
|
||||
|
|
|
@ -59,7 +59,7 @@ import com.google.protobuf.ServiceException;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class RpcRetryingCallerWithReadReplicas {
|
||||
static final Log LOG = LogFactory.getLog(RpcRetryingCallerWithReadReplicas.class);
|
||||
private static final Log LOG = LogFactory.getLog(RpcRetryingCallerWithReadReplicas.class);
|
||||
|
||||
protected final ExecutorService pool;
|
||||
protected final ClusterConnection cConnection;
|
||||
|
|
|
@ -64,6 +64,7 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
|
|||
= "hbase.client.log.scanner.latency.cutoff";
|
||||
public static final String LOG_SCANNER_ACTIVITY = "hbase.client.log.scanner.activity";
|
||||
|
||||
// Keeping LOG public as it is being used in TestScannerHeartbeatMessages
|
||||
public static final Log LOG = LogFactory.getLog(ScannerCallable.class);
|
||||
protected long scannerId = -1L;
|
||||
protected boolean instantiated = false;
|
||||
|
|
|
@ -60,7 +60,7 @@ import com.google.common.annotations.VisibleForTesting;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
|
||||
private final Log LOG = LogFactory.getLog(this.getClass());
|
||||
private static final Log LOG = LogFactory.getLog(ScannerCallableWithReplicas.class);
|
||||
volatile ScannerCallable currentScannerCallable;
|
||||
AtomicBoolean replicaSwitched = new AtomicBoolean(false);
|
||||
final ClusterConnection cConnection;
|
||||
|
|
|
@ -35,7 +35,7 @@ import org.apache.zookeeper.KeeperException;
|
|||
* A cluster registry that stores to zookeeper.
|
||||
*/
|
||||
class ZooKeeperRegistry implements Registry {
|
||||
static final Log LOG = LogFactory.getLog(ZooKeeperRegistry.class);
|
||||
private static final Log LOG = LogFactory.getLog(ZooKeeperRegistry.class);
|
||||
// Needs an instance of hci to function. Set after construct this instance.
|
||||
ConnectionImplementation hci;
|
||||
|
||||
|
|
|
@ -71,7 +71,7 @@ import com.google.protobuf.InvalidProtocolBufferException;
|
|||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Stable
|
||||
public class SingleColumnValueFilter extends FilterBase {
|
||||
static final Log LOG = LogFactory.getLog(SingleColumnValueFilter.class);
|
||||
private static final Log LOG = LogFactory.getLog(SingleColumnValueFilter.class);
|
||||
|
||||
protected byte [] columnFamily;
|
||||
protected byte [] columnQualifier;
|
||||
|
|
|
@ -49,6 +49,7 @@ import java.net.SocketAddress;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public abstract class AbstractRpcClient implements RpcClient {
|
||||
// Log level is being changed in tests
|
||||
public static final Log LOG = LogFactory.getLog(AbstractRpcClient.class);
|
||||
|
||||
protected final Configuration conf;
|
||||
|
@ -287,4 +288,4 @@ public abstract class AbstractRpcClient implements RpcClient {
|
|||
return this.rpcClient.callBlockingMethod(md, pcrc, param, returnType, this.ticket, this.isa);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -39,7 +39,7 @@ import java.io.IOException;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class AsyncCall extends DefaultPromise<Message> {
|
||||
public static final Log LOG = LogFactory.getLog(AsyncCall.class.getName());
|
||||
private static final Log LOG = LogFactory.getLog(AsyncCall.class.getName());
|
||||
|
||||
final int id;
|
||||
|
||||
|
@ -133,4 +133,4 @@ public class AsyncCall extends DefaultPromise<Message> {
|
|||
public long getRpcTimeout() {
|
||||
return rpcTimeout;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -80,7 +80,7 @@ import com.google.protobuf.RpcCallback;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class AsyncRpcChannel {
|
||||
public static final Log LOG = LogFactory.getLog(AsyncRpcChannel.class.getName());
|
||||
private static final Log LOG = LogFactory.getLog(AsyncRpcChannel.class.getName());
|
||||
|
||||
private static final int MAX_SASL_RETRIES = 5;
|
||||
|
||||
|
|
|
@ -43,6 +43,9 @@ import java.util.concurrent.TimeUnit;
|
|||
import java.util.concurrent.TimeoutException;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
|
@ -68,6 +71,8 @@ import com.google.protobuf.RpcController;
|
|||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
|
||||
public class AsyncRpcClient extends AbstractRpcClient {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(AsyncRpcClient.class);
|
||||
|
||||
public static final String CLIENT_MAX_THREADS = "hbase.rpc.client.threads.max";
|
||||
public static final String USE_NATIVE_TRANSPORT = "hbase.rpc.client.nativetransport";
|
||||
public static final String USE_GLOBAL_EVENT_LOOP_GROUP = "hbase.rpc.client.globaleventloopgroup";
|
||||
|
|
|
@ -38,7 +38,7 @@ import com.google.protobuf.Message;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class AsyncServerResponseHandler extends ChannelInboundHandlerAdapter {
|
||||
public static final Log LOG = LogFactory.getLog(AsyncServerResponseHandler.class.getName());
|
||||
private static final Log LOG = LogFactory.getLog(AsyncServerResponseHandler.class.getName());
|
||||
|
||||
private final AsyncRpcChannel channel;
|
||||
|
||||
|
@ -123,4 +123,4 @@ public class AsyncServerResponseHandler extends ChannelInboundHandlerAdapter {
|
|||
e.getPort(), doNotRetry) :
|
||||
new RemoteWithExtrasException(innerExceptionClassName, e.getStackTrace(), doNotRetry);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -43,7 +43,7 @@ import com.google.protobuf.ServiceException;
|
|||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
public abstract class CoprocessorRpcChannel implements RpcChannel, BlockingRpcChannel {
|
||||
private static Log LOG = LogFactory.getLog(CoprocessorRpcChannel.class);
|
||||
private static final Log LOG = LogFactory.getLog(CoprocessorRpcChannel.class);
|
||||
|
||||
@Override
|
||||
@InterfaceAudience.Private
|
||||
|
|
|
@ -53,6 +53,7 @@ import com.google.protobuf.Message;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class IPCUtil {
|
||||
// LOG is being used in TestIPCUtil
|
||||
public static final Log LOG = LogFactory.getLog(IPCUtil.class);
|
||||
/**
|
||||
* How much we think the decompressor will expand the original compressed content.
|
||||
|
|
|
@ -43,7 +43,7 @@ import com.google.protobuf.Message;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class MasterCoprocessorRpcChannel extends CoprocessorRpcChannel{
|
||||
private static Log LOG = LogFactory.getLog(MasterCoprocessorRpcChannel.class);
|
||||
private static final Log LOG = LogFactory.getLog(MasterCoprocessorRpcChannel.class);
|
||||
|
||||
private final HConnection connection;
|
||||
|
||||
|
|
|
@ -47,7 +47,7 @@ import com.google.protobuf.Message;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class RegionCoprocessorRpcChannel extends CoprocessorRpcChannel{
|
||||
private static Log LOG = LogFactory.getLog(RegionCoprocessorRpcChannel.class);
|
||||
private static final Log LOG = LogFactory.getLog(RegionCoprocessorRpcChannel.class);
|
||||
|
||||
private final HConnection connection;
|
||||
private final TableName table;
|
||||
|
|
|
@ -37,7 +37,7 @@ import com.google.protobuf.Message;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class RegionServerCoprocessorRpcChannel extends CoprocessorRpcChannel {
|
||||
private static Log LOG = LogFactory.getLog(RegionServerCoprocessorRpcChannel.class);
|
||||
private static final Log LOG = LogFactory.getLog(RegionServerCoprocessorRpcChannel.class);
|
||||
private final ClusterConnection connection;
|
||||
private final ServerName serverName;
|
||||
|
||||
|
|
|
@ -23,6 +23,9 @@ import com.google.protobuf.Descriptors.MethodDescriptor;
|
|||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.Message.Builder;
|
||||
import com.google.protobuf.RpcCallback;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
|
@ -101,6 +104,7 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class RpcClientImpl extends AbstractRpcClient {
|
||||
private static final Log LOG = LogFactory.getLog(RpcClientImpl.class);
|
||||
protected final AtomicInteger callIdCnt = new AtomicInteger();
|
||||
|
||||
protected final PoolMap<ConnectionId, Connection> connections;
|
||||
|
|
|
@ -63,7 +63,7 @@ import com.google.protobuf.RpcController;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public final class ResponseConverter {
|
||||
public static final Log LOG = LogFactory.getLog(ResponseConverter.class);
|
||||
private static final Log LOG = LogFactory.getLog(ResponseConverter.class);
|
||||
|
||||
private ResponseConverter() {
|
||||
}
|
||||
|
|
|
@ -55,7 +55,7 @@ import com.google.common.annotations.VisibleForTesting;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class HBaseSaslRpcClient {
|
||||
public static final Log LOG = LogFactory.getLog(HBaseSaslRpcClient.class);
|
||||
private static final Log LOG = LogFactory.getLog(HBaseSaslRpcClient.class);
|
||||
|
||||
private final SaslClient saslClient;
|
||||
private final boolean fallbackAllowed;
|
||||
|
|
|
@ -48,7 +48,7 @@ import java.util.Random;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class SaslClientHandler extends ChannelDuplexHandler {
|
||||
public static final Log LOG = LogFactory.getLog(SaslClientHandler.class);
|
||||
private static final Log LOG = LogFactory.getLog(SaslClientHandler.class);
|
||||
|
||||
private final boolean fallbackAllowed;
|
||||
|
||||
|
@ -375,4 +375,4 @@ public class SaslClientHandler extends ChannelDuplexHandler {
|
|||
*/
|
||||
public void onSuccess(Channel channel);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class TablePermission extends Permission {
|
||||
private static Log LOG = LogFactory.getLog(TablePermission.class);
|
||||
private static final Log LOG = LogFactory.getLog(TablePermission.class);
|
||||
|
||||
private TableName table;
|
||||
private byte[] family;
|
||||
|
|
|
@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class UserPermission extends TablePermission {
|
||||
private static Log LOG = LogFactory.getLog(UserPermission.class);
|
||||
private static final Log LOG = LogFactory.getLog(UserPermission.class);
|
||||
|
||||
private byte[] user;
|
||||
|
||||
|
|
|
@ -31,7 +31,7 @@ import org.apache.hadoop.security.token.TokenSelector;
|
|||
@InterfaceAudience.Private
|
||||
public class AuthenticationTokenSelector
|
||||
implements TokenSelector<AuthenticationTokenIdentifier> {
|
||||
private static Log LOG = LogFactory.getLog(AuthenticationTokenSelector.class);
|
||||
private static final Log LOG = LogFactory.getLog(AuthenticationTokenSelector.class);
|
||||
|
||||
public AuthenticationTokenSelector() {
|
||||
}
|
||||
|
|
|
@ -54,7 +54,7 @@ import org.apache.zookeeper.KeeperException;
|
|||
@InterfaceStability.Evolving
|
||||
public class TokenUtil {
|
||||
// This class is referenced indirectly by User out in common; instances are created by reflection
|
||||
private static Log LOG = LogFactory.getLog(TokenUtil.class);
|
||||
private static final Log LOG = LogFactory.getLog(TokenUtil.class);
|
||||
|
||||
/**
|
||||
* Obtain and return an authentication token for the current user.
|
||||
|
|
|
@ -41,7 +41,7 @@ import org.apache.zookeeper.KeeperException;
|
|||
@Deprecated
|
||||
@InterfaceAudience.Private
|
||||
public class ZKLeaderManager extends ZooKeeperListener {
|
||||
private static Log LOG = LogFactory.getLog(ZKLeaderManager.class);
|
||||
private static final Log LOG = LogFactory.getLog(ZKLeaderManager.class);
|
||||
|
||||
private final AtomicBoolean leaderExists = new AtomicBoolean();
|
||||
private String leaderZNode;
|
||||
|
|
|
@ -35,7 +35,8 @@ import org.apache.zookeeper.KeeperException;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public abstract class ZooKeeperNodeTracker extends ZooKeeperListener {
|
||||
static final Log LOG = LogFactory.getLog(ZooKeeperNodeTracker.class);
|
||||
// LOG is being used in subclasses, hence keeping it protected
|
||||
protected static final Log LOG = LogFactory.getLog(ZooKeeperNodeTracker.class);
|
||||
/** Path of node being tracked */
|
||||
protected final String node;
|
||||
|
||||
|
|
|
@ -41,7 +41,7 @@ import org.junit.experimental.categories.Category;
|
|||
*/
|
||||
@Category({MiscTests.class, SmallTests.class})
|
||||
public class TestHTableDescriptor {
|
||||
final static Log LOG = LogFactory.getLog(TestHTableDescriptor.class);
|
||||
private static final Log LOG = LogFactory.getLog(TestHTableDescriptor.class);
|
||||
|
||||
@Test
|
||||
public void testPb() throws DeserializationException, IOException {
|
||||
|
|
|
@ -39,6 +39,8 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
|
@ -65,6 +67,7 @@ import org.mockito.Mockito;
|
|||
|
||||
@Category({ClientTests.class, MediumTests.class})
|
||||
public class TestAsyncProcess {
|
||||
private final static Log LOG = LogFactory.getLog(TestAsyncProcess.class);
|
||||
private static final TableName DUMMY_TABLE =
|
||||
TableName.valueOf("DUMMY_TABLE");
|
||||
private static final byte[] DUMMY_BYTES_1 = "DUMMY_BYTES_1".getBytes();
|
||||
|
|
|
@ -51,7 +51,7 @@ import org.junit.experimental.categories.Category;
|
|||
@Category({ClientTests.class, SmallTests.class})
|
||||
public class TestIPCUtil {
|
||||
|
||||
public static final Log LOG = LogFactory.getLog(IPCUtil.class);
|
||||
private static final Log LOG = LogFactory.getLog(TestIPCUtil.class);
|
||||
|
||||
IPCUtil util;
|
||||
@Before
|
||||
|
|
|
@ -53,7 +53,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ChoreService implements ChoreServicer {
|
||||
private final Log LOG = LogFactory.getLog(this.getClass());
|
||||
private static final Log LOG = LogFactory.getLog(ChoreService.class);
|
||||
|
||||
/**
|
||||
* The minimum number of threads in the core pool of the underlying ScheduledThreadPoolExecutor
|
||||
|
|
|
@ -82,7 +82,7 @@ import com.google.common.annotations.VisibleForTesting;
|
|||
public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId, SettableTimestamp {
|
||||
private static final ArrayList<Tag> EMPTY_ARRAY_LIST = new ArrayList<Tag>();
|
||||
|
||||
static final Log LOG = LogFactory.getLog(KeyValue.class);
|
||||
private static final Log LOG = LogFactory.getLog(KeyValue.class);
|
||||
|
||||
/**
|
||||
* Colon character in UTF-8
|
||||
|
|
|
@ -42,7 +42,7 @@ import com.google.common.annotations.VisibleForTesting;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public abstract class ScheduledChore implements Runnable {
|
||||
private final Log LOG = LogFactory.getLog(this.getClass());
|
||||
private static final Log LOG = LogFactory.getLog(ScheduledChore.class);
|
||||
|
||||
private final String name;
|
||||
|
||||
|
|
|
@ -46,7 +46,7 @@ import com.google.common.annotations.VisibleForTesting;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class BoundedByteBufferPool {
|
||||
private final Log LOG = LogFactory.getLog(this.getClass());
|
||||
private static final Log LOG = LogFactory.getLog(BoundedByteBufferPool.class);
|
||||
|
||||
@VisibleForTesting
|
||||
final Queue<ByteBuffer> buffers;
|
||||
|
@ -110,4 +110,4 @@ public class BoundedByteBufferPool {
|
|||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -47,7 +47,7 @@ import org.apache.hadoop.util.ReflectionUtils;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public final class Compression {
|
||||
static final Log LOG = LogFactory.getLog(Compression.class);
|
||||
private static final Log LOG = LogFactory.getLog(Compression.class);
|
||||
|
||||
/**
|
||||
* Prevent the instantiation of class.
|
||||
|
|
|
@ -34,7 +34,7 @@ import org.apache.hadoop.util.StringUtils;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public final class ByteBufferArray {
|
||||
static final Log LOG = LogFactory.getLog(ByteBufferArray.class);
|
||||
private static final Log LOG = LogFactory.getLog(ByteBufferArray.class);
|
||||
|
||||
static final int DEFAULT_BUFFER_SIZE = 4 * 1024 * 1024;
|
||||
private ByteBuffer buffers[];
|
||||
|
|
|
@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ClassSize {
|
||||
static final Log LOG = LogFactory.getLog(ClassSize.class);
|
||||
private static final Log LOG = LogFactory.getLog(ClassSize.class);
|
||||
|
||||
private static int nrOfRefsPerObj = 2;
|
||||
|
||||
|
|
|
@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
|
||||
@InterfaceAudience.Private
|
||||
public class Methods {
|
||||
private static Log LOG = LogFactory.getLog(Methods.class);
|
||||
private static final Log LOG = LogFactory.getLog(Methods.class);
|
||||
|
||||
public static <T> Object call(Class<T> clazz, T instance, String methodName,
|
||||
Class[] types, Object[] args) throws Exception {
|
||||
|
|
|
@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class Sleeper {
|
||||
private final Log LOG = LogFactory.getLog(this.getClass().getName());
|
||||
private static final Log LOG = LogFactory.getLog(Sleeper.class);
|
||||
private final int period;
|
||||
private final Stoppable stopper;
|
||||
private static final long MINIMAL_DELTA_FOR_LOGGING = 10000;
|
||||
|
|
|
@ -42,7 +42,7 @@ import com.google.common.base.Preconditions;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class Threads {
|
||||
protected static final Log LOG = LogFactory.getLog(Threads.class);
|
||||
private static final Log LOG = LogFactory.getLog(Threads.class);
|
||||
private static final AtomicInteger poolNumber = new AtomicInteger(1);
|
||||
|
||||
private static UncaughtExceptionHandler LOGGING_EXCEPTION_HANDLER =
|
||||
|
|
|
@ -38,7 +38,7 @@ import org.junit.experimental.categories.Category;
|
|||
|
||||
@Category(SmallTests.class)
|
||||
public class TestChoreService {
|
||||
private final Log LOG = LogFactory.getLog(this.getClass());
|
||||
private static final Log LOG = LogFactory.getLog(TestChoreService.class);
|
||||
|
||||
/**
|
||||
* A few ScheduledChore samples that are useful for testing with ChoreService
|
||||
|
|
|
@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
import static org.junit.Assert.assertNotEquals;
|
||||
|
||||
public class TestKeyValue extends TestCase {
|
||||
private final Log LOG = LogFactory.getLog(this.getClass().getName());
|
||||
private static final Log LOG = LogFactory.getLog(TestKeyValue.class);
|
||||
|
||||
public void testColumnCompare() throws Exception {
|
||||
final byte [] a = Bytes.toBytes("aaa");
|
||||
|
|
|
@ -41,7 +41,7 @@ import org.junit.experimental.categories.Category;
|
|||
@Category({MiscTests.class, SmallTests.class})
|
||||
public class TestKeyStoreKeyProvider {
|
||||
|
||||
static final Log LOG = LogFactory.getLog(TestKeyStoreKeyProvider.class);
|
||||
private static final Log LOG = LogFactory.getLog(TestKeyStoreKeyProvider.class);
|
||||
static final HBaseCommonTestingUtility TEST_UTIL = new HBaseCommonTestingUtility();
|
||||
static final String ALIAS = "test";
|
||||
static final String PASSWORD = "password";
|
||||
|
|
|
@ -61,6 +61,7 @@ public class IntegrationTestIngest extends IntegrationTestBase {
|
|||
protected static final String NUM_READ_THREADS_KEY = "num_read_threads";
|
||||
protected static final int DEFAULT_NUM_READ_THREADS = 20;
|
||||
|
||||
// Log is being used in IntegrationTestIngestWithEncryption, hence it is protected
|
||||
protected static final Log LOG = LogFactory.getLog(IntegrationTestIngest.class);
|
||||
protected IntegrationTestingUtility util;
|
||||
protected HBaseCluster cluster;
|
||||
|
|
|
@ -19,6 +19,8 @@ package org.apache.hadoop.hbase;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Waiter.Predicate;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
|
@ -40,7 +42,7 @@ import org.junit.experimental.categories.Category;
|
|||
|
||||
@Category(IntegrationTests.class)
|
||||
public class IntegrationTestIngestWithEncryption extends IntegrationTestIngest {
|
||||
|
||||
private final static Log LOG = LogFactory.getLog(IntegrationTestIngestWithEncryption.class);
|
||||
boolean initialized = false;
|
||||
|
||||
static {
|
||||
|
|
|
@ -42,7 +42,7 @@ import org.junit.experimental.categories.Category;
|
|||
*/
|
||||
@Category(IntegrationTests.class)
|
||||
public class IntegrationTestMetaReplicas {
|
||||
static final Log LOG = LogFactory.getLog(IntegrationTestMetaReplicas.class);
|
||||
private static final Log LOG = LogFactory.getLog(IntegrationTestMetaReplicas.class);
|
||||
/**
|
||||
* Util to get at the cluster.
|
||||
*/
|
||||
|
|
|
@ -50,7 +50,7 @@ public class Action {
|
|||
public static final String KILL_RS_TIMEOUT_KEY = "hbase.chaosmonkey.action.killrstimeout";
|
||||
public static final String START_RS_TIMEOUT_KEY = "hbase.chaosmonkey.action.startrstimeout";
|
||||
|
||||
protected static Log LOG = LogFactory.getLog(Action.class);
|
||||
protected static final Log LOG = LogFactory.getLog(Action.class);
|
||||
|
||||
protected static final long KILL_MASTER_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
|
||||
protected static final long START_MASTER_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
|
||||
|
|
|
@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
|
|||
* limits the maximum number of servers that can be down at the same time during rolling restarts.
|
||||
*/
|
||||
public class RollingBatchRestartRsAction extends BatchRestartRsAction {
|
||||
private static Log LOG = LogFactory.getLog(RollingBatchRestartRsAction.class);
|
||||
private static final Log LOG = LogFactory.getLog(RollingBatchRestartRsAction.class);
|
||||
protected int maxDeadServers; // number of maximum dead servers at any given time. Defaults to 5
|
||||
|
||||
public RollingBatchRestartRsAction(long sleepTime, float ratio) {
|
||||
|
|
|
@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.util.StoppableImplementation;
|
|||
*/
|
||||
public abstract class Policy extends StoppableImplementation implements Runnable {
|
||||
|
||||
protected static Log LOG = LogFactory.getLog(Policy.class);
|
||||
protected static final Log LOG = LogFactory.getLog(Policy.class);
|
||||
|
||||
protected PolicyContext context;
|
||||
|
||||
|
|
|
@ -63,7 +63,7 @@ import org.junit.experimental.categories.Category;
|
|||
public class IntegrationTestImportTsv implements Configurable, Tool {
|
||||
|
||||
private static final String NAME = IntegrationTestImportTsv.class.getSimpleName();
|
||||
protected static final Log LOG = LogFactory.getLog(IntegrationTestImportTsv.class);
|
||||
private static final Log LOG = LogFactory.getLog(IntegrationTestImportTsv.class);
|
||||
|
||||
protected static final String simple_tsv =
|
||||
"row1\t1\tc1\tc2\n" +
|
||||
|
|
|
@ -36,7 +36,7 @@ import org.apache.log4j.Logger;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class RESTServlet implements Constants {
|
||||
private static Logger LOG = Logger.getLogger(RESTServlet.class);
|
||||
private static final Logger LOG = Logger.getLogger(RESTServlet.class);
|
||||
private static RESTServlet INSTANCE;
|
||||
private final Configuration conf;
|
||||
private final MetricsREST metrics = new MetricsREST();
|
||||
|
|
|
@ -32,7 +32,7 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
|
||||
public class DummyFilter implements Filter {
|
||||
private Log LOG = LogFactory.getLog(getClass());
|
||||
private static final Log LOG = LogFactory.getLog(DummyFilter.class);
|
||||
|
||||
@Override
|
||||
public void destroy() {
|
||||
|
|
|
@ -33,7 +33,7 @@ import com.sun.jersey.spi.container.servlet.ServletContainer;
|
|||
|
||||
public class HBaseRESTTestingUtility {
|
||||
|
||||
static final Log LOG = LogFactory.getLog(HBaseRESTTestingUtility.class);
|
||||
private static final Log LOG = LogFactory.getLog(HBaseRESTTestingUtility.class);
|
||||
|
||||
private int testServletPort;
|
||||
private Server server;
|
||||
|
|
|
@ -47,7 +47,7 @@ import org.junit.experimental.categories.Category;
|
|||
|
||||
@Category({RestTests.class, MediumTests.class})
|
||||
public class TestStatusResource {
|
||||
public static Log LOG = LogFactory.getLog(TestStatusResource.class);
|
||||
private static final Log LOG = LogFactory.getLog(TestStatusResource.class);
|
||||
|
||||
private static final byte[] META_REGION_NAME = Bytes.toBytes(TableName.META_TABLE_NAME + ",,1");
|
||||
|
||||
|
|
|
@ -29,7 +29,7 @@ import org.apache.hadoop.util.StringUtils;
|
|||
* The Class HealthCheckChore for running health checker regularly.
|
||||
*/
|
||||
public class HealthCheckChore extends ScheduledChore {
|
||||
private static Log LOG = LogFactory.getLog(HealthCheckChore.class);
|
||||
private static final Log LOG = LogFactory.getLog(HealthCheckChore.class);
|
||||
private HealthChecker healthChecker;
|
||||
private Configuration config;
|
||||
private int threshold;
|
||||
|
|
|
@ -35,7 +35,7 @@ import org.apache.hadoop.util.Shell.ShellCommandExecutor;
|
|||
*/
|
||||
class HealthChecker {
|
||||
|
||||
private static Log LOG = LogFactory.getLog(HealthChecker.class);
|
||||
private static final Log LOG = LogFactory.getLog(HealthChecker.class);
|
||||
private ShellCommandExecutor shexec = null;
|
||||
private String exceptionStackTrace;
|
||||
|
||||
|
|
|
@ -48,7 +48,7 @@ import javax.rmi.ssl.SslRMIServerSocketFactory;
|
|||
*/
|
||||
public class JMXListener implements Coprocessor {
|
||||
|
||||
public static final Log LOG = LogFactory.getLog(JMXListener.class);
|
||||
private static final Log LOG = LogFactory.getLog(JMXListener.class);
|
||||
public static final String RMI_REGISTRY_PORT_CONF_KEY = ".rmi.registry.port";
|
||||
public static final String RMI_CONNECTOR_PORT_CONF_KEY = ".rmi.connector.port";
|
||||
public static final int defMasterRMIRegistryPort = 10101;
|
||||
|
|
|
@ -60,7 +60,7 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil;
|
|||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
public class LocalHBaseCluster {
|
||||
static final Log LOG = LogFactory.getLog(LocalHBaseCluster.class);
|
||||
private static final Log LOG = LogFactory.getLog(LocalHBaseCluster.class);
|
||||
private final List<JVMClusterUtil.MasterThread> masterThreads =
|
||||
new CopyOnWriteArrayList<JVMClusterUtil.MasterThread>();
|
||||
private final List<JVMClusterUtil.RegionServerThread> regionThreads =
|
||||
|
|
|
@ -48,7 +48,7 @@ import java.util.concurrent.ConcurrentSkipListMap;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ZKNamespaceManager extends ZooKeeperListener {
|
||||
private static Log LOG = LogFactory.getLog(ZKNamespaceManager.class);
|
||||
private static final Log LOG = LogFactory.getLog(ZKNamespaceManager.class);
|
||||
private final String nsZNode;
|
||||
private volatile NavigableMap<String,NamespaceDescriptor> cache;
|
||||
|
||||
|
|
|
@ -45,7 +45,7 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
|||
* check its content to make sure that the backup server is not now in charge.</p>
|
||||
*/
|
||||
public class ZNodeClearer {
|
||||
public static final Log LOG = LogFactory.getLog(ZNodeClearer.class);
|
||||
private static final Log LOG = LogFactory.getLog(ZNodeClearer.class);
|
||||
|
||||
private ZNodeClearer() {}
|
||||
|
||||
|
|
|
@ -73,7 +73,7 @@ import java.util.WeakHashMap;
|
|||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
public class ConfigurationManager {
|
||||
public static final Log LOG = LogFactory.getLog(ConfigurationManager.class);
|
||||
private static final Log LOG = LogFactory.getLog(ConfigurationManager.class);
|
||||
|
||||
// The set of Configuration Observers. These classes would like to get
|
||||
// notified when the configuration is reloaded from disk. This is a set
|
||||
|
|
|
@ -40,7 +40,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ForeignExceptionDispatcher implements ForeignExceptionListener, ForeignExceptionSnare {
|
||||
public static final Log LOG = LogFactory.getLog(ForeignExceptionDispatcher.class);
|
||||
private static final Log LOG = LogFactory.getLog(ForeignExceptionDispatcher.class);
|
||||
protected final String name;
|
||||
protected final List<ForeignExceptionListener> listeners =
|
||||
new ArrayList<ForeignExceptionListener>();
|
||||
|
|
|
@ -33,7 +33,7 @@ import org.mortbay.jetty.RequestLog;
|
|||
*/
|
||||
public class HttpRequestLog {
|
||||
|
||||
public static final Log LOG = LogFactory.getLog(HttpRequestLog.class);
|
||||
private static final Log LOG = LogFactory.getLog(HttpRequestLog.class);
|
||||
private static final HashMap<String, String> serverToComponent;
|
||||
|
||||
static {
|
||||
|
|
|
@ -101,7 +101,7 @@ import com.sun.jersey.spi.container.servlet.ServletContainer;
|
|||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
public class HttpServer implements FilterContainer {
|
||||
public static final Log LOG = LogFactory.getLog(HttpServer.class);
|
||||
private static final Log LOG = LogFactory.getLog(HttpServer.class);
|
||||
|
||||
static final String FILTER_INITIALIZERS_PROPERTY
|
||||
= "hbase.http.filter.initializers";
|
||||
|
|
|
@ -51,7 +51,7 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class HalfStoreFileReader extends StoreFile.Reader {
|
||||
final Log LOG = LogFactory.getLog(HalfStoreFileReader.class);
|
||||
private static final Log LOG = LogFactory.getLog(HalfStoreFileReader.class);
|
||||
final boolean top;
|
||||
// This is the key we split around. Its the first possible entry on a row:
|
||||
// i.e. empty column and a timestamp of LATEST_TIMESTAMP.
|
||||
|
|
|
@ -135,6 +135,7 @@ import com.google.common.base.Preconditions;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class HFile {
|
||||
// LOG is being used in HFileBlock and CheckSumUtil
|
||||
static final Log LOG = LogFactory.getLog(HFile.class);
|
||||
|
||||
/**
|
||||
|
|
|
@ -97,7 +97,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|||
@JsonIgnoreProperties({"encodingCountsForTest"})
|
||||
public class LruBlockCache implements ResizableBlockCache, HeapSize {
|
||||
|
||||
static final Log LOG = LogFactory.getLog(LruBlockCache.class);
|
||||
private static final Log LOG = LogFactory.getLog(LruBlockCache.class);
|
||||
|
||||
/**
|
||||
* Percentage of total size that eviction will evict until; e.g. if set to .8, then we will keep
|
||||
|
|
|
@ -49,7 +49,7 @@ import com.google.common.primitives.Ints;
|
|||
@InterfaceAudience.Private
|
||||
@JsonIgnoreProperties({"indexStatistics", "freeSize", "usedSize"})
|
||||
public final class BucketAllocator {
|
||||
static final Log LOG = LogFactory.getLog(BucketAllocator.class);
|
||||
private static final Log LOG = LogFactory.getLog(BucketAllocator.class);
|
||||
|
||||
@JsonIgnoreProperties({"completelyFree", "uninstantiated"})
|
||||
public final static class Bucket {
|
||||
|
|
|
@ -92,7 +92,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class BucketCache implements BlockCache, HeapSize {
|
||||
static final Log LOG = LogFactory.getLog(BucketCache.class);
|
||||
private static final Log LOG = LogFactory.getLog(BucketCache.class);
|
||||
|
||||
/** Priority buckets */
|
||||
private static final float DEFAULT_SINGLE_FACTOR = 0.25f;
|
||||
|
|
|
@ -18,6 +18,8 @@ package org.apache.hadoop.hbase.ipc;
|
|||
*/
|
||||
import java.nio.channels.ClosedChannelException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.ipc.RpcServer.Call;
|
||||
|
@ -38,6 +40,8 @@ import com.google.protobuf.Message;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class CallRunner {
|
||||
private static final Log LOG = LogFactory.getLog(CallRunner.class);
|
||||
|
||||
private Call call;
|
||||
private RpcServerInterface rpcServer;
|
||||
private MonitoredRPCHandler status;
|
||||
|
|
|
@ -157,6 +157,7 @@ import com.google.protobuf.TextFormat;
|
|||
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
|
||||
@InterfaceStability.Evolving
|
||||
public class RpcServer implements RpcServerInterface {
|
||||
// LOG is being used in CallRunner and the log level is being changed in tests
|
||||
public static final Log LOG = LogFactory.getLog(RpcServer.class);
|
||||
private static final CallQueueTooBigException CALL_QUEUE_TOO_BIG_EXCEPTION
|
||||
= new CallQueueTooBigException();
|
||||
|
|
|
@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.util.BoundedPriorityBlockingQueue;
|
|||
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
|
||||
@InterfaceStability.Evolving
|
||||
public class SimpleRpcScheduler extends RpcScheduler {
|
||||
public static final Log LOG = LogFactory.getLog(SimpleRpcScheduler.class);
|
||||
private static final Log LOG = LogFactory.getLog(SimpleRpcScheduler.class);
|
||||
|
||||
public static final String CALL_QUEUE_READ_SHARE_CONF_KEY =
|
||||
"hbase.ipc.server.callqueue.read.ratio";
|
||||
|
|
|
@ -44,7 +44,7 @@ import static org.apache.hadoop.hbase.mapreduce.TableRecordReaderImpl.LOG_PER_RO
|
|||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Stable
|
||||
public class TableRecordReaderImpl {
|
||||
static final Log LOG = LogFactory.getLog(TableRecordReaderImpl.class);
|
||||
private static final Log LOG = LogFactory.getLog(TableRecordReaderImpl.class);
|
||||
|
||||
private byte [] startRow;
|
||||
private byte [] endRow;
|
||||
|
|
|
@ -57,7 +57,7 @@ import com.google.common.annotations.VisibleForTesting;
|
|||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Stable
|
||||
public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable, KeyValue> {
|
||||
static Log LOG = LogFactory.getLog(HFileOutputFormat.class);
|
||||
private static final Log LOG = LogFactory.getLog(HFileOutputFormat.class);
|
||||
|
||||
// This constant is public since the client can modify this when setting
|
||||
// up their conf object and thus refer to this symbol.
|
||||
|
|
|
@ -86,7 +86,7 @@ import com.google.common.annotations.VisibleForTesting;
|
|||
@InterfaceStability.Evolving
|
||||
public class HFileOutputFormat2
|
||||
extends FileOutputFormat<ImmutableBytesWritable, Cell> {
|
||||
static Log LOG = LogFactory.getLog(HFileOutputFormat2.class);
|
||||
private static final Log LOG = LogFactory.getLog(HFileOutputFormat2.class);
|
||||
|
||||
// The following constants are private since these are used by
|
||||
// HFileOutputFormat2 to internally transfer data between job setup and
|
||||
|
|
|
@ -59,7 +59,7 @@ import java.util.Iterator;
|
|||
public abstract class MultiTableInputFormatBase extends
|
||||
InputFormat<ImmutableBytesWritable, Result> {
|
||||
|
||||
final Log LOG = LogFactory.getLog(MultiTableInputFormatBase.class);
|
||||
private static final Log LOG = LogFactory.getLog(MultiTableInputFormatBase.class);
|
||||
|
||||
/** Holds the set of scans used to define the input. */
|
||||
private List<Scan> scans;
|
||||
|
|
|
@ -113,7 +113,7 @@ extends InputFormat<ImmutableBytesWritable, Result> {
|
|||
* default is true. False means the table is using binary row key*/
|
||||
public static final String TABLE_ROW_TEXTKEY = "hbase.table.row.textkey";
|
||||
|
||||
final Log LOG = LogFactory.getLog(TableInputFormatBase.class);
|
||||
private static final Log LOG = LogFactory.getLog(TableInputFormatBase.class);
|
||||
|
||||
private static final String NOT_INITIALIZED = "The input format instance has not been properly " +
|
||||
"initialized. Ensure you call initializeTable either in your constructor or initialize " +
|
||||
|
|
|
@ -70,7 +70,7 @@ import com.yammer.metrics.core.MetricsRegistry;
|
|||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Stable
|
||||
public class TableMapReduceUtil {
|
||||
static Log LOG = LogFactory.getLog(TableMapReduceUtil.class);
|
||||
private static final Log LOG = LogFactory.getLog(TableMapReduceUtil.class);
|
||||
|
||||
/**
|
||||
* Use this before submitting a TableMap job. It will appropriately set up
|
||||
|
|
|
@ -49,7 +49,7 @@ public class TableRecordReaderImpl {
|
|||
public static final String LOG_PER_ROW_COUNT
|
||||
= "hbase.mapreduce.log.scanner.rowcount";
|
||||
|
||||
static final Log LOG = LogFactory.getLog(TableRecordReaderImpl.class);
|
||||
private static final Log LOG = LogFactory.getLog(TableRecordReaderImpl.class);
|
||||
|
||||
// HBASE_COUNTER_GROUP_NAME is the name of mapreduce counter group for HBase
|
||||
private static final String HBASE_COUNTER_GROUP_NAME =
|
||||
|
|
|
@ -43,6 +43,8 @@ import org.apache.hadoop.mapreduce.InputSplit;
|
|||
@InterfaceStability.Evolving
|
||||
public class TableSplit extends InputSplit
|
||||
implements Writable, Comparable<TableSplit> {
|
||||
/** @deprecated LOG variable would be made private. */
|
||||
@Deprecated
|
||||
public static final Log LOG = LogFactory.getLog(TableSplit.class);
|
||||
|
||||
// should be < 0 (@see #readFields(DataInput))
|
||||
|
|
|
@ -68,7 +68,7 @@ import java.util.TreeMap;
|
|||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Stable
|
||||
public class WALPlayer extends Configured implements Tool {
|
||||
final static Log LOG = LogFactory.getLog(WALPlayer.class);
|
||||
private static final Log LOG = LogFactory.getLog(WALPlayer.class);
|
||||
final static String NAME = "WALPlayer";
|
||||
final static String BULK_OUTPUT_CONF_KEY = "wal.bulk.output";
|
||||
final static String TABLES_KEY = "wal.input.tables";
|
||||
|
|
|
@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.master.balancer.FavoredNodesPlan;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class AssignmentVerificationReport {
|
||||
protected static final Log LOG = LogFactory.getLog(
|
||||
private static final Log LOG = LogFactory.getLog(
|
||||
AssignmentVerificationReport.class.getName());
|
||||
|
||||
private TableName tableName = null;
|
||||
|
|
|
@ -181,7 +181,7 @@ import com.google.protobuf.ServiceException;
|
|||
@SuppressWarnings("deprecation")
|
||||
public class MasterRpcServices extends RSRpcServices
|
||||
implements MasterService.BlockingInterface, RegionServerStatusService.BlockingInterface {
|
||||
protected static final Log LOG = LogFactory.getLog(MasterRpcServices.class.getName());
|
||||
private static final Log LOG = LogFactory.getLog(MasterRpcServices.class.getName());
|
||||
|
||||
private final HMaster master;
|
||||
|
||||
|
|
|
@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.master.MetricsMasterWrapper;
|
|||
@InterfaceStability.Evolving
|
||||
@InterfaceAudience.Private
|
||||
public class MetricsMaster {
|
||||
private final Log LOG = LogFactory.getLog(this.getClass());
|
||||
private static final Log LOG = LogFactory.getLog(MetricsMaster.class);
|
||||
private MetricsMasterSource masterSource;
|
||||
|
||||
public MetricsMaster(MetricsMasterWrapper masterWrapper) {
|
||||
|
|
|
@ -36,7 +36,7 @@ import org.apache.hadoop.net.ScriptBasedMapping;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class RackManager {
|
||||
static final Log LOG = LogFactory.getLog(RackManager.class);
|
||||
private static final Log LOG = LogFactory.getLog(RackManager.class);
|
||||
public static final String UNKNOWN_RACK = "Unknown Rack";
|
||||
|
||||
private DNSToSwitchMapping switchMapping;
|
||||
|
|
|
@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.ServerName;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class FavoredNodesPlan {
|
||||
protected static final Log LOG = LogFactory.getLog(
|
||||
private static final Log LOG = LogFactory.getLog(
|
||||
FavoredNodesPlan.class.getName());
|
||||
|
||||
/** the map between each region and its favored region server list */
|
||||
|
|
|
@ -51,7 +51,7 @@ import com.google.common.cache.LoadingCache;
|
|||
*/
|
||||
class RegionLocationFinder {
|
||||
|
||||
private static Log LOG = LogFactory.getLog(RegionLocationFinder.class);
|
||||
private static final Log LOG = LogFactory.getLog(RegionLocationFinder.class);
|
||||
|
||||
private Configuration conf;
|
||||
private ClusterStatus status;
|
||||
|
|
|
@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.wal.DefaultWALProvider;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class LogCleaner extends CleanerChore<BaseLogCleanerDelegate> {
|
||||
static final Log LOG = LogFactory.getLog(LogCleaner.class.getName());
|
||||
private static final Log LOG = LogFactory.getLog(LogCleaner.class.getName());
|
||||
|
||||
/**
|
||||
* @param p the period of time to sleep between each run
|
||||
|
|
|
@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
|||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
|
||||
public class TimeToLiveHFileCleaner extends BaseHFileCleanerDelegate {
|
||||
|
||||
public static final Log LOG = LogFactory.getLog(TimeToLiveHFileCleaner.class.getName());
|
||||
private static final Log LOG = LogFactory.getLog(TimeToLiveHFileCleaner.class.getName());
|
||||
public static final String TTL_CONF_KEY = "hbase.master.hfilecleaner.ttl";
|
||||
// default ttl = 5 minutes
|
||||
public static final long DEFAULT_TTL = 60000 * 5;
|
||||
|
|
|
@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
|||
*/
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
|
||||
public class TimeToLiveLogCleaner extends BaseLogCleanerDelegate {
|
||||
static final Log LOG = LogFactory.getLog(TimeToLiveLogCleaner.class.getName());
|
||||
private static final Log LOG = LogFactory.getLog(TimeToLiveLogCleaner.class.getName());
|
||||
// Configured time a log can be kept after it was closed
|
||||
private long ttl;
|
||||
private boolean stopped = false;
|
||||
|
|
|
@ -40,7 +40,7 @@ import com.google.common.annotations.VisibleForTesting;
|
|||
*/
|
||||
@InterfaceAudience.Public
|
||||
public class NamespaceAuditor {
|
||||
private static Log LOG = LogFactory.getLog(NamespaceAuditor.class);
|
||||
private static final Log LOG = LogFactory.getLog(NamespaceAuditor.class);
|
||||
static final String NS_AUDITOR_INIT_TIMEOUT = "hbase.namespace.auditor.init.timeout";
|
||||
static final int DEFAULT_NS_AUDITOR_INIT_TIMEOUT = 120000;
|
||||
private NamespaceStateManager stateManager;
|
||||
|
|
|
@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
@InterfaceAudience.Private
|
||||
class NamespaceStateManager {
|
||||
|
||||
private static Log LOG = LogFactory.getLog(NamespaceStateManager.class);
|
||||
private static final Log LOG = LogFactory.getLog(NamespaceStateManager.class);
|
||||
private ConcurrentMap<String, NamespaceTableAndRegionInfo> nsStateCache;
|
||||
private MasterServices master;
|
||||
private volatile boolean initialized = false;
|
||||
|
|
|
@ -38,7 +38,7 @@ import com.google.protobuf.InvalidProtocolBufferException;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ZKProcedureCoordinatorRpcs implements ProcedureCoordinatorRpcs {
|
||||
public static final Log LOG = LogFactory.getLog(ZKProcedureCoordinatorRpcs.class);
|
||||
private static final Log LOG = LogFactory.getLog(ZKProcedureCoordinatorRpcs.class);
|
||||
private ZKProcedureUtil zkProc = null;
|
||||
protected ProcedureCoordinator coordinator = null; // if started this should be non-null
|
||||
|
||||
|
|
|
@ -69,7 +69,7 @@ import com.google.protobuf.TextFormat;
|
|||
//to figure out whether it is a meta region or not.
|
||||
@InterfaceAudience.Private
|
||||
class AnnotationReadingPriorityFunction implements PriorityFunction {
|
||||
public static final Log LOG =
|
||||
private static final Log LOG =
|
||||
LogFactory.getLog(AnnotationReadingPriorityFunction.class.getName());
|
||||
|
||||
/** Used to control the scan delay, currently sqrt(numNextCall * weight) */
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue