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:
Usha Kuchibhotla 2015-04-28 16:54:36 +05:30 committed by Sean Busbey
parent 84dd84fc58
commit 94a00bea07
287 changed files with 330 additions and 289 deletions

View File

@ -96,7 +96,7 @@ import com.google.common.annotations.VisibleForTesting;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
class AsyncProcess { 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(); protected static final AtomicLong COUNTER = new AtomicLong();
public static final String PRIMARY_CALL_TIMEOUT_KEY = "hbase.client.primaryCallTimeout.multiget"; public static final String PRIMARY_CALL_TIMEOUT_KEY = "hbase.client.primaryCallTimeout.multiget";

View File

@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.util.Bytes;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
final class ClientIdGenerator { final class ClientIdGenerator {
static final Log LOG = LogFactory.getLog(ClientIdGenerator.class); private static final Log LOG = LogFactory.getLog(ClientIdGenerator.class);
private ClientIdGenerator() {} private ClientIdGenerator() {}

View File

@ -54,7 +54,7 @@ import com.google.common.annotations.VisibleForTesting;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class ClientScanner extends AbstractClientScanner { 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 // A byte array in which all elements are the max byte, and it is used to
// construct closest front row // construct closest front row
static byte[] MAX_BYTE_ARRAY = Bytes.createMaxByteArray(9); static byte[] MAX_BYTE_ARRAY = Bytes.createMaxByteArray(9);

View File

@ -51,7 +51,7 @@ import com.google.protobuf.ServiceException;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class ClientSmallScanner extends ClientScanner { 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 ScannerCallableWithReplicas smallScanCallable = null;
private SmallScannerCallableFactory callableFactory; private SmallScannerCallableFactory callableFactory;

View File

@ -98,7 +98,7 @@ import java.util.concurrent.atomic.AtomicInteger;
@InterfaceAudience.Private @InterfaceAudience.Private
class ConnectionImplementation implements ClusterConnection, Closeable { class ConnectionImplementation implements ClusterConnection, Closeable {
public static final String RETRIES_BY_SERVER_KEY = "hbase.client.retries.by.server"; 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 static final String CLIENT_NONCES_ENABLED_KEY = "hbase.client.nonces.enabled";
private final long pause; private final long pause;

View File

@ -73,7 +73,7 @@ import org.apache.hadoop.ipc.RemoteException;
@InterfaceAudience.Private @InterfaceAudience.Private
class PreemptiveFastFailInterceptor extends RetryingCallerInterceptor { class PreemptiveFastFailInterceptor extends RetryingCallerInterceptor {
public static final Log LOG = LogFactory private static final Log LOG = LogFactory
.getLog(PreemptiveFastFailInterceptor.class); .getLog(PreemptiveFastFailInterceptor.class);
// amount of time to wait before we consider a server to be in fast fail // amount of time to wait before we consider a server to be in fast fail

View File

@ -47,7 +47,7 @@ import org.apache.hadoop.hbase.util.Bytes;
@InterfaceAudience.Private @InterfaceAudience.Private
public abstract class RegionServerCallable<T> implements RetryingCallable<T> { public abstract class RegionServerCallable<T> implements RetryingCallable<T> {
// Public because used outside of this package over in ipc. // 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 Connection connection;
protected final TableName tableName; protected final TableName tableName;
protected final byte[] row; protected final byte[] row;

View File

@ -49,6 +49,7 @@ import com.google.protobuf.ServiceException;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class RpcRetryingCallerImpl<T> implements RpcRetryingCaller<T> { 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); public static final Log LOG = LogFactory.getLog(RpcRetryingCallerImpl.class);
/** /**
* When we started making calls. * When we started making calls.

View File

@ -59,7 +59,7 @@ import com.google.protobuf.ServiceException;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class RpcRetryingCallerWithReadReplicas { 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 ExecutorService pool;
protected final ClusterConnection cConnection; protected final ClusterConnection cConnection;

View File

@ -64,6 +64,7 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
= "hbase.client.log.scanner.latency.cutoff"; = "hbase.client.log.scanner.latency.cutoff";
public static final String LOG_SCANNER_ACTIVITY = "hbase.client.log.scanner.activity"; 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); public static final Log LOG = LogFactory.getLog(ScannerCallable.class);
protected long scannerId = -1L; protected long scannerId = -1L;
protected boolean instantiated = false; protected boolean instantiated = false;

View File

@ -60,7 +60,7 @@ import com.google.common.annotations.VisibleForTesting;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
class ScannerCallableWithReplicas implements RetryingCallable<Result[]> { 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; volatile ScannerCallable currentScannerCallable;
AtomicBoolean replicaSwitched = new AtomicBoolean(false); AtomicBoolean replicaSwitched = new AtomicBoolean(false);
final ClusterConnection cConnection; final ClusterConnection cConnection;

View File

@ -35,7 +35,7 @@ import org.apache.zookeeper.KeeperException;
* A cluster registry that stores to zookeeper. * A cluster registry that stores to zookeeper.
*/ */
class ZooKeeperRegistry implements Registry { 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. // Needs an instance of hci to function. Set after construct this instance.
ConnectionImplementation hci; ConnectionImplementation hci;

View File

@ -71,7 +71,7 @@ import com.google.protobuf.InvalidProtocolBufferException;
@InterfaceAudience.Public @InterfaceAudience.Public
@InterfaceStability.Stable @InterfaceStability.Stable
public class SingleColumnValueFilter extends FilterBase { 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 [] columnFamily;
protected byte [] columnQualifier; protected byte [] columnQualifier;

View File

@ -49,6 +49,7 @@ import java.net.SocketAddress;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public abstract class AbstractRpcClient implements RpcClient { public abstract class AbstractRpcClient implements RpcClient {
// Log level is being changed in tests
public static final Log LOG = LogFactory.getLog(AbstractRpcClient.class); public static final Log LOG = LogFactory.getLog(AbstractRpcClient.class);
protected final Configuration conf; 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); return this.rpcClient.callBlockingMethod(md, pcrc, param, returnType, this.ticket, this.isa);
} }
} }
} }

View File

@ -39,7 +39,7 @@ import java.io.IOException;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class AsyncCall extends DefaultPromise<Message> { 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; final int id;
@ -133,4 +133,4 @@ public class AsyncCall extends DefaultPromise<Message> {
public long getRpcTimeout() { public long getRpcTimeout() {
return rpcTimeout; return rpcTimeout;
} }
} }

View File

@ -80,7 +80,7 @@ import com.google.protobuf.RpcCallback;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class AsyncRpcChannel { 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; private static final int MAX_SASL_RETRIES = 5;

View File

@ -43,6 +43,9 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException; import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger; 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.conf.Configuration;
import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HBaseInterfaceAudience;
@ -68,6 +71,8 @@ import com.google.protobuf.RpcController;
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG) @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
public class AsyncRpcClient extends AbstractRpcClient { 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 CLIENT_MAX_THREADS = "hbase.rpc.client.threads.max";
public static final String USE_NATIVE_TRANSPORT = "hbase.rpc.client.nativetransport"; public static final String USE_NATIVE_TRANSPORT = "hbase.rpc.client.nativetransport";
public static final String USE_GLOBAL_EVENT_LOOP_GROUP = "hbase.rpc.client.globaleventloopgroup"; public static final String USE_GLOBAL_EVENT_LOOP_GROUP = "hbase.rpc.client.globaleventloopgroup";

View File

@ -38,7 +38,7 @@ import com.google.protobuf.Message;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class AsyncServerResponseHandler extends ChannelInboundHandlerAdapter { 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; private final AsyncRpcChannel channel;
@ -123,4 +123,4 @@ public class AsyncServerResponseHandler extends ChannelInboundHandlerAdapter {
e.getPort(), doNotRetry) : e.getPort(), doNotRetry) :
new RemoteWithExtrasException(innerExceptionClassName, e.getStackTrace(), doNotRetry); new RemoteWithExtrasException(innerExceptionClassName, e.getStackTrace(), doNotRetry);
} }
} }

View File

@ -43,7 +43,7 @@ import com.google.protobuf.ServiceException;
@InterfaceAudience.Public @InterfaceAudience.Public
@InterfaceStability.Evolving @InterfaceStability.Evolving
public abstract class CoprocessorRpcChannel implements RpcChannel, BlockingRpcChannel { 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 @Override
@InterfaceAudience.Private @InterfaceAudience.Private

View File

@ -53,6 +53,7 @@ import com.google.protobuf.Message;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class IPCUtil { public class IPCUtil {
// LOG is being used in TestIPCUtil
public static final Log LOG = LogFactory.getLog(IPCUtil.class); public static final Log LOG = LogFactory.getLog(IPCUtil.class);
/** /**
* How much we think the decompressor will expand the original compressed content. * How much we think the decompressor will expand the original compressed content.

View File

@ -43,7 +43,7 @@ import com.google.protobuf.Message;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class MasterCoprocessorRpcChannel extends CoprocessorRpcChannel{ 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; private final HConnection connection;

View File

@ -47,7 +47,7 @@ import com.google.protobuf.Message;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class RegionCoprocessorRpcChannel extends CoprocessorRpcChannel{ 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 HConnection connection;
private final TableName table; private final TableName table;

View File

@ -37,7 +37,7 @@ import com.google.protobuf.Message;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class RegionServerCoprocessorRpcChannel extends CoprocessorRpcChannel { 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 ClusterConnection connection;
private final ServerName serverName; private final ServerName serverName;

View File

@ -23,6 +23,9 @@ import com.google.protobuf.Descriptors.MethodDescriptor;
import com.google.protobuf.Message; import com.google.protobuf.Message;
import com.google.protobuf.Message.Builder; import com.google.protobuf.Message.Builder;
import com.google.protobuf.RpcCallback; 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.conf.Configuration;
import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.DoNotRetryIOException;
@ -101,6 +104,7 @@ import java.util.concurrent.atomic.AtomicInteger;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class RpcClientImpl extends AbstractRpcClient { public class RpcClientImpl extends AbstractRpcClient {
private static final Log LOG = LogFactory.getLog(RpcClientImpl.class);
protected final AtomicInteger callIdCnt = new AtomicInteger(); protected final AtomicInteger callIdCnt = new AtomicInteger();
protected final PoolMap<ConnectionId, Connection> connections; protected final PoolMap<ConnectionId, Connection> connections;

View File

@ -63,7 +63,7 @@ import com.google.protobuf.RpcController;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public final class ResponseConverter { public final class ResponseConverter {
public static final Log LOG = LogFactory.getLog(ResponseConverter.class); private static final Log LOG = LogFactory.getLog(ResponseConverter.class);
private ResponseConverter() { private ResponseConverter() {
} }

View File

@ -55,7 +55,7 @@ import com.google.common.annotations.VisibleForTesting;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class HBaseSaslRpcClient { 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 SaslClient saslClient;
private final boolean fallbackAllowed; private final boolean fallbackAllowed;

View File

@ -48,7 +48,7 @@ import java.util.Random;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class SaslClientHandler extends ChannelDuplexHandler { 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; private final boolean fallbackAllowed;
@ -375,4 +375,4 @@ public class SaslClientHandler extends ChannelDuplexHandler {
*/ */
public void onSuccess(Channel channel); public void onSuccess(Channel channel);
} }
} }

View File

@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.util.Bytes;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class TablePermission extends Permission { 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 TableName table;
private byte[] family; private byte[] family;

View File

@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.util.Bytes;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class UserPermission extends TablePermission { 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; private byte[] user;

View File

@ -31,7 +31,7 @@ import org.apache.hadoop.security.token.TokenSelector;
@InterfaceAudience.Private @InterfaceAudience.Private
public class AuthenticationTokenSelector public class AuthenticationTokenSelector
implements TokenSelector<AuthenticationTokenIdentifier> { implements TokenSelector<AuthenticationTokenIdentifier> {
private static Log LOG = LogFactory.getLog(AuthenticationTokenSelector.class); private static final Log LOG = LogFactory.getLog(AuthenticationTokenSelector.class);
public AuthenticationTokenSelector() { public AuthenticationTokenSelector() {
} }

View File

@ -54,7 +54,7 @@ import org.apache.zookeeper.KeeperException;
@InterfaceStability.Evolving @InterfaceStability.Evolving
public class TokenUtil { public class TokenUtil {
// This class is referenced indirectly by User out in common; instances are created by reflection // 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. * Obtain and return an authentication token for the current user.

View File

@ -41,7 +41,7 @@ import org.apache.zookeeper.KeeperException;
@Deprecated @Deprecated
@InterfaceAudience.Private @InterfaceAudience.Private
public class ZKLeaderManager extends ZooKeeperListener { 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 final AtomicBoolean leaderExists = new AtomicBoolean();
private String leaderZNode; private String leaderZNode;

View File

@ -35,7 +35,8 @@ import org.apache.zookeeper.KeeperException;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public abstract class ZooKeeperNodeTracker extends ZooKeeperListener { 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 */ /** Path of node being tracked */
protected final String node; protected final String node;

View File

@ -41,7 +41,7 @@ import org.junit.experimental.categories.Category;
*/ */
@Category({MiscTests.class, SmallTests.class}) @Category({MiscTests.class, SmallTests.class})
public class TestHTableDescriptor { public class TestHTableDescriptor {
final static Log LOG = LogFactory.getLog(TestHTableDescriptor.class); private static final Log LOG = LogFactory.getLog(TestHTableDescriptor.class);
@Test @Test
public void testPb() throws DeserializationException, IOException { public void testPb() throws DeserializationException, IOException {

View File

@ -39,6 +39,8 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong; 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.conf.Configuration;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
@ -65,6 +67,7 @@ import org.mockito.Mockito;
@Category({ClientTests.class, MediumTests.class}) @Category({ClientTests.class, MediumTests.class})
public class TestAsyncProcess { public class TestAsyncProcess {
private final static Log LOG = LogFactory.getLog(TestAsyncProcess.class);
private static final TableName DUMMY_TABLE = private static final TableName DUMMY_TABLE =
TableName.valueOf("DUMMY_TABLE"); TableName.valueOf("DUMMY_TABLE");
private static final byte[] DUMMY_BYTES_1 = "DUMMY_BYTES_1".getBytes(); private static final byte[] DUMMY_BYTES_1 = "DUMMY_BYTES_1".getBytes();

View File

@ -51,7 +51,7 @@ import org.junit.experimental.categories.Category;
@Category({ClientTests.class, SmallTests.class}) @Category({ClientTests.class, SmallTests.class})
public class TestIPCUtil { public class TestIPCUtil {
public static final Log LOG = LogFactory.getLog(IPCUtil.class); private static final Log LOG = LogFactory.getLog(TestIPCUtil.class);
IPCUtil util; IPCUtil util;
@Before @Before

View File

@ -53,7 +53,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class ChoreService implements ChoreServicer { 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 * The minimum number of threads in the core pool of the underlying ScheduledThreadPoolExecutor

View File

@ -82,7 +82,7 @@ import com.google.common.annotations.VisibleForTesting;
public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId, SettableTimestamp { public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId, SettableTimestamp {
private static final ArrayList<Tag> EMPTY_ARRAY_LIST = new ArrayList<Tag>(); 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 * Colon character in UTF-8

View File

@ -42,7 +42,7 @@ import com.google.common.annotations.VisibleForTesting;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public abstract class ScheduledChore implements Runnable { 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; private final String name;

View File

@ -46,7 +46,7 @@ import com.google.common.annotations.VisibleForTesting;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class BoundedByteBufferPool { public class BoundedByteBufferPool {
private final Log LOG = LogFactory.getLog(this.getClass()); private static final Log LOG = LogFactory.getLog(BoundedByteBufferPool.class);
@VisibleForTesting @VisibleForTesting
final Queue<ByteBuffer> buffers; final Queue<ByteBuffer> buffers;
@ -110,4 +110,4 @@ public class BoundedByteBufferPool {
} }
} }
} }
} }

View File

@ -47,7 +47,7 @@ import org.apache.hadoop.util.ReflectionUtils;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public final class Compression { 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. * Prevent the instantiation of class.

View File

@ -34,7 +34,7 @@ import org.apache.hadoop.util.StringUtils;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public final class ByteBufferArray { 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; static final int DEFAULT_BUFFER_SIZE = 4 * 1024 * 1024;
private ByteBuffer buffers[]; private ByteBuffer buffers[];

View File

@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class ClassSize { 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; private static int nrOfRefsPerObj = 2;

View File

@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
@InterfaceAudience.Private @InterfaceAudience.Private
public class Methods { 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, public static <T> Object call(Class<T> clazz, T instance, String methodName,
Class[] types, Object[] args) throws Exception { Class[] types, Object[] args) throws Exception {

View File

@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class Sleeper { 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 int period;
private final Stoppable stopper; private final Stoppable stopper;
private static final long MINIMAL_DELTA_FOR_LOGGING = 10000; private static final long MINIMAL_DELTA_FOR_LOGGING = 10000;

View File

@ -42,7 +42,7 @@ import com.google.common.base.Preconditions;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class Threads { 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 final AtomicInteger poolNumber = new AtomicInteger(1);
private static UncaughtExceptionHandler LOGGING_EXCEPTION_HANDLER = private static UncaughtExceptionHandler LOGGING_EXCEPTION_HANDLER =

View File

@ -38,7 +38,7 @@ import org.junit.experimental.categories.Category;
@Category(SmallTests.class) @Category(SmallTests.class)
public class TestChoreService { 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 * A few ScheduledChore samples that are useful for testing with ChoreService

View File

@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.util.Bytes;
import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotEquals;
public class TestKeyValue extends TestCase { 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 { public void testColumnCompare() throws Exception {
final byte [] a = Bytes.toBytes("aaa"); final byte [] a = Bytes.toBytes("aaa");

View File

@ -41,7 +41,7 @@ import org.junit.experimental.categories.Category;
@Category({MiscTests.class, SmallTests.class}) @Category({MiscTests.class, SmallTests.class})
public class TestKeyStoreKeyProvider { 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 HBaseCommonTestingUtility TEST_UTIL = new HBaseCommonTestingUtility();
static final String ALIAS = "test"; static final String ALIAS = "test";
static final String PASSWORD = "password"; static final String PASSWORD = "password";

View File

@ -61,6 +61,7 @@ public class IntegrationTestIngest extends IntegrationTestBase {
protected static final String NUM_READ_THREADS_KEY = "num_read_threads"; protected static final String NUM_READ_THREADS_KEY = "num_read_threads";
protected static final int DEFAULT_NUM_READ_THREADS = 20; 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 static final Log LOG = LogFactory.getLog(IntegrationTestIngest.class);
protected IntegrationTestingUtility util; protected IntegrationTestingUtility util;
protected HBaseCluster cluster; protected HBaseCluster cluster;

View File

@ -19,6 +19,8 @@ package org.apache.hadoop.hbase;
import java.io.IOException; 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.conf.Configuration;
import org.apache.hadoop.hbase.Waiter.Predicate; import org.apache.hadoop.hbase.Waiter.Predicate;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
@ -40,7 +42,7 @@ import org.junit.experimental.categories.Category;
@Category(IntegrationTests.class) @Category(IntegrationTests.class)
public class IntegrationTestIngestWithEncryption extends IntegrationTestIngest { public class IntegrationTestIngestWithEncryption extends IntegrationTestIngest {
private final static Log LOG = LogFactory.getLog(IntegrationTestIngestWithEncryption.class);
boolean initialized = false; boolean initialized = false;
static { static {

View File

@ -42,7 +42,7 @@ import org.junit.experimental.categories.Category;
*/ */
@Category(IntegrationTests.class) @Category(IntegrationTests.class)
public class IntegrationTestMetaReplicas { 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. * Util to get at the cluster.
*/ */

View File

@ -50,7 +50,7 @@ public class Action {
public static final String KILL_RS_TIMEOUT_KEY = "hbase.chaosmonkey.action.killrstimeout"; public static final String KILL_RS_TIMEOUT_KEY = "hbase.chaosmonkey.action.killrstimeout";
public static final String START_RS_TIMEOUT_KEY = "hbase.chaosmonkey.action.startrstimeout"; 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 KILL_MASTER_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
protected static final long START_MASTER_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT; protected static final long START_MASTER_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;

View File

@ -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. * limits the maximum number of servers that can be down at the same time during rolling restarts.
*/ */
public class RollingBatchRestartRsAction extends BatchRestartRsAction { 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 protected int maxDeadServers; // number of maximum dead servers at any given time. Defaults to 5
public RollingBatchRestartRsAction(long sleepTime, float ratio) { public RollingBatchRestartRsAction(long sleepTime, float ratio) {

View File

@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.util.StoppableImplementation;
*/ */
public abstract class Policy extends StoppableImplementation implements Runnable { 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; protected PolicyContext context;

View File

@ -63,7 +63,7 @@ import org.junit.experimental.categories.Category;
public class IntegrationTestImportTsv implements Configurable, Tool { public class IntegrationTestImportTsv implements Configurable, Tool {
private static final String NAME = IntegrationTestImportTsv.class.getSimpleName(); 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 = protected static final String simple_tsv =
"row1\t1\tc1\tc2\n" + "row1\t1\tc1\tc2\n" +

View File

@ -36,7 +36,7 @@ import org.apache.log4j.Logger;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class RESTServlet implements Constants { 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 static RESTServlet INSTANCE;
private final Configuration conf; private final Configuration conf;
private final MetricsREST metrics = new MetricsREST(); private final MetricsREST metrics = new MetricsREST();

View File

@ -32,7 +32,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
public class DummyFilter implements Filter { public class DummyFilter implements Filter {
private Log LOG = LogFactory.getLog(getClass()); private static final Log LOG = LogFactory.getLog(DummyFilter.class);
@Override @Override
public void destroy() { public void destroy() {

View File

@ -33,7 +33,7 @@ import com.sun.jersey.spi.container.servlet.ServletContainer;
public class HBaseRESTTestingUtility { public class HBaseRESTTestingUtility {
static final Log LOG = LogFactory.getLog(HBaseRESTTestingUtility.class); private static final Log LOG = LogFactory.getLog(HBaseRESTTestingUtility.class);
private int testServletPort; private int testServletPort;
private Server server; private Server server;

View File

@ -47,7 +47,7 @@ import org.junit.experimental.categories.Category;
@Category({RestTests.class, MediumTests.class}) @Category({RestTests.class, MediumTests.class})
public class TestStatusResource { 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"); private static final byte[] META_REGION_NAME = Bytes.toBytes(TableName.META_TABLE_NAME + ",,1");

View File

@ -29,7 +29,7 @@ import org.apache.hadoop.util.StringUtils;
* The Class HealthCheckChore for running health checker regularly. * The Class HealthCheckChore for running health checker regularly.
*/ */
public class HealthCheckChore extends ScheduledChore { 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 HealthChecker healthChecker;
private Configuration config; private Configuration config;
private int threshold; private int threshold;

View File

@ -35,7 +35,7 @@ import org.apache.hadoop.util.Shell.ShellCommandExecutor;
*/ */
class HealthChecker { class HealthChecker {
private static Log LOG = LogFactory.getLog(HealthChecker.class); private static final Log LOG = LogFactory.getLog(HealthChecker.class);
private ShellCommandExecutor shexec = null; private ShellCommandExecutor shexec = null;
private String exceptionStackTrace; private String exceptionStackTrace;

View File

@ -48,7 +48,7 @@ import javax.rmi.ssl.SslRMIServerSocketFactory;
*/ */
public class JMXListener implements Coprocessor { 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_REGISTRY_PORT_CONF_KEY = ".rmi.registry.port";
public static final String RMI_CONNECTOR_PORT_CONF_KEY = ".rmi.connector.port"; public static final String RMI_CONNECTOR_PORT_CONF_KEY = ".rmi.connector.port";
public static final int defMasterRMIRegistryPort = 10101; public static final int defMasterRMIRegistryPort = 10101;

View File

@ -60,7 +60,7 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil;
@InterfaceAudience.Public @InterfaceAudience.Public
@InterfaceStability.Evolving @InterfaceStability.Evolving
public class LocalHBaseCluster { 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 = private final List<JVMClusterUtil.MasterThread> masterThreads =
new CopyOnWriteArrayList<JVMClusterUtil.MasterThread>(); new CopyOnWriteArrayList<JVMClusterUtil.MasterThread>();
private final List<JVMClusterUtil.RegionServerThread> regionThreads = private final List<JVMClusterUtil.RegionServerThread> regionThreads =

View File

@ -48,7 +48,7 @@ import java.util.concurrent.ConcurrentSkipListMap;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class ZKNamespaceManager extends ZooKeeperListener { 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 final String nsZNode;
private volatile NavigableMap<String,NamespaceDescriptor> cache; private volatile NavigableMap<String,NamespaceDescriptor> cache;

View File

@ -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> * check its content to make sure that the backup server is not now in charge.</p>
*/ */
public class ZNodeClearer { public class ZNodeClearer {
public static final Log LOG = LogFactory.getLog(ZNodeClearer.class); private static final Log LOG = LogFactory.getLog(ZNodeClearer.class);
private ZNodeClearer() {} private ZNodeClearer() {}

View File

@ -73,7 +73,7 @@ import java.util.WeakHashMap;
@InterfaceAudience.Private @InterfaceAudience.Private
@InterfaceStability.Evolving @InterfaceStability.Evolving
public class ConfigurationManager { 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 // The set of Configuration Observers. These classes would like to get
// notified when the configuration is reloaded from disk. This is a set // notified when the configuration is reloaded from disk. This is a set

View File

@ -40,7 +40,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class ForeignExceptionDispatcher implements ForeignExceptionListener, ForeignExceptionSnare { 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 String name;
protected final List<ForeignExceptionListener> listeners = protected final List<ForeignExceptionListener> listeners =
new ArrayList<ForeignExceptionListener>(); new ArrayList<ForeignExceptionListener>();

View File

@ -33,7 +33,7 @@ import org.mortbay.jetty.RequestLog;
*/ */
public class HttpRequestLog { 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; private static final HashMap<String, String> serverToComponent;
static { static {

View File

@ -101,7 +101,7 @@ import com.sun.jersey.spi.container.servlet.ServletContainer;
@InterfaceAudience.Private @InterfaceAudience.Private
@InterfaceStability.Evolving @InterfaceStability.Evolving
public class HttpServer implements FilterContainer { 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 static final String FILTER_INITIALIZERS_PROPERTY
= "hbase.http.filter.initializers"; = "hbase.http.filter.initializers";

View File

@ -51,7 +51,7 @@ import org.apache.hadoop.hbase.util.Bytes;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class HalfStoreFileReader extends StoreFile.Reader { 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; final boolean top;
// This is the key we split around. Its the first possible entry on a row: // 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. // i.e. empty column and a timestamp of LATEST_TIMESTAMP.

View File

@ -135,6 +135,7 @@ import com.google.common.base.Preconditions;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class HFile { public class HFile {
// LOG is being used in HFileBlock and CheckSumUtil
static final Log LOG = LogFactory.getLog(HFile.class); static final Log LOG = LogFactory.getLog(HFile.class);
/** /**

View File

@ -97,7 +97,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
@JsonIgnoreProperties({"encodingCountsForTest"}) @JsonIgnoreProperties({"encodingCountsForTest"})
public class LruBlockCache implements ResizableBlockCache, HeapSize { 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 * Percentage of total size that eviction will evict until; e.g. if set to .8, then we will keep

View File

@ -49,7 +49,7 @@ import com.google.common.primitives.Ints;
@InterfaceAudience.Private @InterfaceAudience.Private
@JsonIgnoreProperties({"indexStatistics", "freeSize", "usedSize"}) @JsonIgnoreProperties({"indexStatistics", "freeSize", "usedSize"})
public final class BucketAllocator { public final class BucketAllocator {
static final Log LOG = LogFactory.getLog(BucketAllocator.class); private static final Log LOG = LogFactory.getLog(BucketAllocator.class);
@JsonIgnoreProperties({"completelyFree", "uninstantiated"}) @JsonIgnoreProperties({"completelyFree", "uninstantiated"})
public final static class Bucket { public final static class Bucket {

View File

@ -92,7 +92,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class BucketCache implements BlockCache, HeapSize { 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 */ /** Priority buckets */
private static final float DEFAULT_SINGLE_FACTOR = 0.25f; private static final float DEFAULT_SINGLE_FACTOR = 0.25f;

View File

@ -18,6 +18,8 @@ package org.apache.hadoop.hbase.ipc;
*/ */
import java.nio.channels.ClosedChannelException; 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.CellScanner;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.ipc.RpcServer.Call; import org.apache.hadoop.hbase.ipc.RpcServer.Call;
@ -38,6 +40,8 @@ import com.google.protobuf.Message;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class CallRunner { public class CallRunner {
private static final Log LOG = LogFactory.getLog(CallRunner.class);
private Call call; private Call call;
private RpcServerInterface rpcServer; private RpcServerInterface rpcServer;
private MonitoredRPCHandler status; private MonitoredRPCHandler status;

View File

@ -157,6 +157,7 @@ import com.google.protobuf.TextFormat;
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX}) @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
@InterfaceStability.Evolving @InterfaceStability.Evolving
public class RpcServer implements RpcServerInterface { 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); public static final Log LOG = LogFactory.getLog(RpcServer.class);
private static final CallQueueTooBigException CALL_QUEUE_TOO_BIG_EXCEPTION private static final CallQueueTooBigException CALL_QUEUE_TOO_BIG_EXCEPTION
= new CallQueueTooBigException(); = new CallQueueTooBigException();

View File

@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.util.BoundedPriorityBlockingQueue;
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX}) @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
@InterfaceStability.Evolving @InterfaceStability.Evolving
public class SimpleRpcScheduler extends RpcScheduler { 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 = public static final String CALL_QUEUE_READ_SHARE_CONF_KEY =
"hbase.ipc.server.callqueue.read.ratio"; "hbase.ipc.server.callqueue.read.ratio";

View File

@ -44,7 +44,7 @@ import static org.apache.hadoop.hbase.mapreduce.TableRecordReaderImpl.LOG_PER_RO
@InterfaceAudience.Public @InterfaceAudience.Public
@InterfaceStability.Stable @InterfaceStability.Stable
public class TableRecordReaderImpl { 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 [] startRow;
private byte [] endRow; private byte [] endRow;

View File

@ -57,7 +57,7 @@ import com.google.common.annotations.VisibleForTesting;
@InterfaceAudience.Public @InterfaceAudience.Public
@InterfaceStability.Stable @InterfaceStability.Stable
public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable, KeyValue> { 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 // This constant is public since the client can modify this when setting
// up their conf object and thus refer to this symbol. // up their conf object and thus refer to this symbol.

View File

@ -86,7 +86,7 @@ import com.google.common.annotations.VisibleForTesting;
@InterfaceStability.Evolving @InterfaceStability.Evolving
public class HFileOutputFormat2 public class HFileOutputFormat2
extends FileOutputFormat<ImmutableBytesWritable, Cell> { 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 // The following constants are private since these are used by
// HFileOutputFormat2 to internally transfer data between job setup and // HFileOutputFormat2 to internally transfer data between job setup and

View File

@ -59,7 +59,7 @@ import java.util.Iterator;
public abstract class MultiTableInputFormatBase extends public abstract class MultiTableInputFormatBase extends
InputFormat<ImmutableBytesWritable, Result> { 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. */ /** Holds the set of scans used to define the input. */
private List<Scan> scans; private List<Scan> scans;

View File

@ -113,7 +113,7 @@ extends InputFormat<ImmutableBytesWritable, Result> {
* default is true. False means the table is using binary row key*/ * default is true. False means the table is using binary row key*/
public static final String TABLE_ROW_TEXTKEY = "hbase.table.row.textkey"; 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 " + 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 " + "initialized. Ensure you call initializeTable either in your constructor or initialize " +

View File

@ -70,7 +70,7 @@ import com.yammer.metrics.core.MetricsRegistry;
@InterfaceAudience.Public @InterfaceAudience.Public
@InterfaceStability.Stable @InterfaceStability.Stable
public class TableMapReduceUtil { 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 * Use this before submitting a TableMap job. It will appropriately set up

View File

@ -49,7 +49,7 @@ public class TableRecordReaderImpl {
public static final String LOG_PER_ROW_COUNT public static final String LOG_PER_ROW_COUNT
= "hbase.mapreduce.log.scanner.rowcount"; = "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 // HBASE_COUNTER_GROUP_NAME is the name of mapreduce counter group for HBase
private static final String HBASE_COUNTER_GROUP_NAME = private static final String HBASE_COUNTER_GROUP_NAME =

View File

@ -43,6 +43,8 @@ import org.apache.hadoop.mapreduce.InputSplit;
@InterfaceStability.Evolving @InterfaceStability.Evolving
public class TableSplit extends InputSplit public class TableSplit extends InputSplit
implements Writable, Comparable<TableSplit> { implements Writable, Comparable<TableSplit> {
/** @deprecated LOG variable would be made private. */
@Deprecated
public static final Log LOG = LogFactory.getLog(TableSplit.class); public static final Log LOG = LogFactory.getLog(TableSplit.class);
// should be < 0 (@see #readFields(DataInput)) // should be < 0 (@see #readFields(DataInput))

View File

@ -68,7 +68,7 @@ import java.util.TreeMap;
@InterfaceAudience.Public @InterfaceAudience.Public
@InterfaceStability.Stable @InterfaceStability.Stable
public class WALPlayer extends Configured implements Tool { 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 NAME = "WALPlayer";
final static String BULK_OUTPUT_CONF_KEY = "wal.bulk.output"; final static String BULK_OUTPUT_CONF_KEY = "wal.bulk.output";
final static String TABLES_KEY = "wal.input.tables"; final static String TABLES_KEY = "wal.input.tables";

View File

@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.master.balancer.FavoredNodesPlan;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class AssignmentVerificationReport { public class AssignmentVerificationReport {
protected static final Log LOG = LogFactory.getLog( private static final Log LOG = LogFactory.getLog(
AssignmentVerificationReport.class.getName()); AssignmentVerificationReport.class.getName());
private TableName tableName = null; private TableName tableName = null;

View File

@ -181,7 +181,7 @@ import com.google.protobuf.ServiceException;
@SuppressWarnings("deprecation") @SuppressWarnings("deprecation")
public class MasterRpcServices extends RSRpcServices public class MasterRpcServices extends RSRpcServices
implements MasterService.BlockingInterface, RegionServerStatusService.BlockingInterface { 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; private final HMaster master;

View File

@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.master.MetricsMasterWrapper;
@InterfaceStability.Evolving @InterfaceStability.Evolving
@InterfaceAudience.Private @InterfaceAudience.Private
public class MetricsMaster { public class MetricsMaster {
private final Log LOG = LogFactory.getLog(this.getClass()); private static final Log LOG = LogFactory.getLog(MetricsMaster.class);
private MetricsMasterSource masterSource; private MetricsMasterSource masterSource;
public MetricsMaster(MetricsMasterWrapper masterWrapper) { public MetricsMaster(MetricsMasterWrapper masterWrapper) {

View File

@ -36,7 +36,7 @@ import org.apache.hadoop.net.ScriptBasedMapping;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class RackManager { 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"; public static final String UNKNOWN_RACK = "Unknown Rack";
private DNSToSwitchMapping switchMapping; private DNSToSwitchMapping switchMapping;

View File

@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.ServerName;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class FavoredNodesPlan { public class FavoredNodesPlan {
protected static final Log LOG = LogFactory.getLog( private static final Log LOG = LogFactory.getLog(
FavoredNodesPlan.class.getName()); FavoredNodesPlan.class.getName());
/** the map between each region and its favored region server list */ /** the map between each region and its favored region server list */

View File

@ -51,7 +51,7 @@ import com.google.common.cache.LoadingCache;
*/ */
class RegionLocationFinder { class RegionLocationFinder {
private static Log LOG = LogFactory.getLog(RegionLocationFinder.class); private static final Log LOG = LogFactory.getLog(RegionLocationFinder.class);
private Configuration conf; private Configuration conf;
private ClusterStatus status; private ClusterStatus status;

View File

@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.wal.DefaultWALProvider;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class LogCleaner extends CleanerChore<BaseLogCleanerDelegate> { 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 * @param p the period of time to sleep between each run

View File

@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG) @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
public class TimeToLiveHFileCleaner extends BaseHFileCleanerDelegate { 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"; public static final String TTL_CONF_KEY = "hbase.master.hfilecleaner.ttl";
// default ttl = 5 minutes // default ttl = 5 minutes
public static final long DEFAULT_TTL = 60000 * 5; public static final long DEFAULT_TTL = 60000 * 5;

View File

@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
*/ */
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG) @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
public class TimeToLiveLogCleaner extends BaseLogCleanerDelegate { 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 // Configured time a log can be kept after it was closed
private long ttl; private long ttl;
private boolean stopped = false; private boolean stopped = false;

View File

@ -40,7 +40,7 @@ import com.google.common.annotations.VisibleForTesting;
*/ */
@InterfaceAudience.Public @InterfaceAudience.Public
public class NamespaceAuditor { 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 String NS_AUDITOR_INIT_TIMEOUT = "hbase.namespace.auditor.init.timeout";
static final int DEFAULT_NS_AUDITOR_INIT_TIMEOUT = 120000; static final int DEFAULT_NS_AUDITOR_INIT_TIMEOUT = 120000;
private NamespaceStateManager stateManager; private NamespaceStateManager stateManager;

View File

@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.util.Bytes;
@InterfaceAudience.Private @InterfaceAudience.Private
class NamespaceStateManager { 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 ConcurrentMap<String, NamespaceTableAndRegionInfo> nsStateCache;
private MasterServices master; private MasterServices master;
private volatile boolean initialized = false; private volatile boolean initialized = false;

View File

@ -38,7 +38,7 @@ import com.google.protobuf.InvalidProtocolBufferException;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class ZKProcedureCoordinatorRpcs implements ProcedureCoordinatorRpcs { 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; private ZKProcedureUtil zkProc = null;
protected ProcedureCoordinator coordinator = null; // if started this should be non-null protected ProcedureCoordinator coordinator = null; // if started this should be non-null

View File

@ -69,7 +69,7 @@ import com.google.protobuf.TextFormat;
//to figure out whether it is a meta region or not. //to figure out whether it is a meta region or not.
@InterfaceAudience.Private @InterfaceAudience.Private
class AnnotationReadingPriorityFunction implements PriorityFunction { class AnnotationReadingPriorityFunction implements PriorityFunction {
public static final Log LOG = private static final Log LOG =
LogFactory.getLog(AnnotationReadingPriorityFunction.class.getName()); LogFactory.getLog(AnnotationReadingPriorityFunction.class.getName());
/** Used to control the scan delay, currently sqrt(numNextCall * weight) */ /** 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