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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

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

View File

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

View File

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

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

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.
*/
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) {

View File

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

View File

@ -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" +

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

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>
*/
public class ZNodeClearer {
public static final Log LOG = LogFactory.getLog(ZNodeClearer.class);
private static final Log LOG = LogFactory.getLog(ZNodeClearer.class);
private ZNodeClearer() {}

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -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 " +

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -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 */

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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