[SPARK-17332][CORE] Make Java Loggers static members

## What changes were proposed in this pull request?

Make all Java Loggers static members

## How was this patch tested?

Jenkins

Author: Sean Owen <sowen@cloudera.com>

Closes #14896 from srowen/SPARK-17332.
This commit is contained in:
Sean Owen 2016-08-31 11:09:14 -07:00 committed by Marcelo Vanzin
parent 9bcb33c541
commit 5d84c7fd83
27 changed files with 29 additions and 27 deletions

View file

@ -56,7 +56,7 @@ import org.apache.spark.network.util.TransportFrameDecoder;
* processes to send messages back to the client on an existing channel.
*/
public class TransportContext {
private final Logger logger = LoggerFactory.getLogger(TransportContext.class);
private static final Logger logger = LoggerFactory.getLogger(TransportContext.class);
private final TransportConf conf;
private final RpcHandler rpcHandler;

View file

@ -72,7 +72,7 @@ import static org.apache.spark.network.util.NettyUtils.getRemoteAddress;
* Concurrency: thread safe and can be called from multiple threads.
*/
public class TransportClient implements Closeable {
private final Logger logger = LoggerFactory.getLogger(TransportClient.class);
private static final Logger logger = LoggerFactory.getLogger(TransportClient.class);
private final Channel channel;
private final TransportResponseHandler handler;

View file

@ -73,7 +73,7 @@ public class TransportClientFactory implements Closeable {
}
}
private final Logger logger = LoggerFactory.getLogger(TransportClientFactory.class);
private static final Logger logger = LoggerFactory.getLogger(TransportClientFactory.class);
private final TransportContext context;
private final TransportConf conf;

View file

@ -48,7 +48,7 @@ import org.apache.spark.network.util.TransportFrameDecoder;
* Concurrency: thread safe and can be called from multiple threads.
*/
public class TransportResponseHandler extends MessageHandler<ResponseMessage> {
private final Logger logger = LoggerFactory.getLogger(TransportResponseHandler.class);
private static final Logger logger = LoggerFactory.getLogger(TransportResponseHandler.class);
private final Channel channel;

View file

@ -33,7 +33,8 @@ import org.slf4j.LoggerFactory;
@ChannelHandler.Sharable
public final class MessageDecoder extends MessageToMessageDecoder<ByteBuf> {
private final Logger logger = LoggerFactory.getLogger(MessageDecoder.class);
private static final Logger logger = LoggerFactory.getLogger(MessageDecoder.class);
@Override
public void decode(ChannelHandlerContext ctx, ByteBuf in, List<Object> out) {
Message.Type msgType = Message.Type.decode(in);

View file

@ -33,7 +33,7 @@ import org.slf4j.LoggerFactory;
@ChannelHandler.Sharable
public final class MessageEncoder extends MessageToMessageEncoder<Message> {
private final Logger logger = LoggerFactory.getLogger(MessageEncoder.class);
private static final Logger logger = LoggerFactory.getLogger(MessageEncoder.class);
/***
* Encodes a Message by invoking its encode() method. For non-data messages, we will add one

View file

@ -38,7 +38,7 @@ import org.apache.spark.network.util.TransportConf;
* server should be setup with a {@link SaslRpcHandler} with matching keys for the given appId.
*/
public class SaslClientBootstrap implements TransportClientBootstrap {
private final Logger logger = LoggerFactory.getLogger(SaslClientBootstrap.class);
private static final Logger logger = LoggerFactory.getLogger(SaslClientBootstrap.class);
private final boolean encrypt;
private final TransportConf conf;

View file

@ -43,7 +43,7 @@ import static org.apache.spark.network.sasl.SparkSaslServer.*;
* firstToken, which is then followed by a set of challenges and responses.
*/
public class SparkSaslClient implements SaslEncryptionBackend {
private final Logger logger = LoggerFactory.getLogger(SparkSaslClient.class);
private static final Logger logger = LoggerFactory.getLogger(SparkSaslClient.class);
private final String secretKeyId;
private final SecretKeyHolder secretKeyHolder;

View file

@ -45,7 +45,7 @@ import org.slf4j.LoggerFactory;
* connections on some socket.)
*/
public class SparkSaslServer implements SaslEncryptionBackend {
private final Logger logger = LoggerFactory.getLogger(SparkSaslServer.class);
private static final Logger logger = LoggerFactory.getLogger(SparkSaslServer.class);
/**
* This is passed as the server name when creating the sasl client/server.

View file

@ -36,7 +36,7 @@ import org.apache.spark.network.client.TransportClient;
* individually fetched as chunks by the client. Each registered buffer is one chunk.
*/
public class OneForOneStreamManager extends StreamManager {
private final Logger logger = LoggerFactory.getLogger(OneForOneStreamManager.class);
private static final Logger logger = LoggerFactory.getLogger(OneForOneStreamManager.class);
private final AtomicLong nextStreamId;
private final ConcurrentHashMap<Long, StreamState> streams;

View file

@ -83,7 +83,7 @@ public abstract class RpcHandler {
private static class OneWayRpcCallback implements RpcResponseCallback {
private final Logger logger = LoggerFactory.getLogger(OneWayRpcCallback.class);
private static final Logger logger = LoggerFactory.getLogger(OneWayRpcCallback.class);
@Override
public void onSuccess(ByteBuffer response) {

View file

@ -49,7 +49,7 @@ import static org.apache.spark.network.util.NettyUtils.getRemoteAddress;
* timeout if the client is continuously sending but getting no responses, for simplicity.
*/
public class TransportChannelHandler extends SimpleChannelInboundHandler<Message> {
private final Logger logger = LoggerFactory.getLogger(TransportChannelHandler.class);
private static final Logger logger = LoggerFactory.getLogger(TransportChannelHandler.class);
private final TransportClient client;
private final TransportResponseHandler responseHandler;

View file

@ -53,7 +53,7 @@ import static org.apache.spark.network.util.NettyUtils.getRemoteAddress;
* The messages should have been processed by the pipeline setup by {@link TransportServer}.
*/
public class TransportRequestHandler extends MessageHandler<RequestMessage> {
private final Logger logger = LoggerFactory.getLogger(TransportRequestHandler.class);
private static final Logger logger = LoggerFactory.getLogger(TransportRequestHandler.class);
/** The Netty channel that this handler is associated with. */
private final Channel channel;

View file

@ -44,7 +44,7 @@ import org.apache.spark.network.util.TransportConf;
* Server for the efficient, low-level streaming service.
*/
public class TransportServer implements Closeable {
private final Logger logger = LoggerFactory.getLogger(TransportServer.class);
private static final Logger logger = LoggerFactory.getLogger(TransportServer.class);
private final TransportContext context;
private final TransportConf conf;

View file

@ -29,7 +29,8 @@ import org.apache.spark.network.util.JavaUtils;
* A class that manages shuffle secret used by the external shuffle service.
*/
public class ShuffleSecretManager implements SecretKeyHolder {
private final Logger logger = LoggerFactory.getLogger(ShuffleSecretManager.class);
private static final Logger logger = LoggerFactory.getLogger(ShuffleSecretManager.class);
private final ConcurrentHashMap<String, String> shuffleSecretMap;
// Spark user used for authenticating SASL connections

View file

@ -54,7 +54,7 @@ import org.apache.spark.network.util.TransportConf;
* level shuffle block.
*/
public class ExternalShuffleBlockHandler extends RpcHandler {
private final Logger logger = LoggerFactory.getLogger(ExternalShuffleBlockHandler.class);
private static final Logger logger = LoggerFactory.getLogger(ExternalShuffleBlockHandler.class);
@VisibleForTesting
final ExternalShuffleBlockResolver blockManager;

View file

@ -44,7 +44,7 @@ import org.apache.spark.network.util.TransportConf;
* executors.
*/
public class ExternalShuffleClient extends ShuffleClient {
private final Logger logger = LoggerFactory.getLogger(ExternalShuffleClient.class);
private static final Logger logger = LoggerFactory.getLogger(ExternalShuffleClient.class);
private final TransportConf conf;
private final boolean saslEnabled;

View file

@ -41,7 +41,7 @@ import org.apache.spark.network.shuffle.protocol.StreamHandle;
* {@link org.apache.spark.network.server.OneForOneStreamManager} on the server side.
*/
public class OneForOneBlockFetcher {
private final Logger logger = LoggerFactory.getLogger(OneForOneBlockFetcher.class);
private static final Logger logger = LoggerFactory.getLogger(OneForOneBlockFetcher.class);
private final TransportClient client;
private final OpenBlocks openMessage;

View file

@ -64,7 +64,7 @@ public class RetryingBlockFetcher {
private static final ExecutorService executorService = Executors.newCachedThreadPool(
NettyUtils.createThreadFactory("Block Fetch Retry"));
private final Logger logger = LoggerFactory.getLogger(RetryingBlockFetcher.class);
private static final Logger logger = LoggerFactory.getLogger(RetryingBlockFetcher.class);
/** Used to initiate new Block Fetches on our remaining blocks. */
private final BlockFetchStarter fetchStarter;

View file

@ -44,7 +44,7 @@ import org.apache.spark.network.util.TransportConf;
* has to detect this itself.
*/
public class MesosExternalShuffleClient extends ExternalShuffleClient {
private final Logger logger = LoggerFactory.getLogger(MesosExternalShuffleClient.class);
private static final Logger logger = LoggerFactory.getLogger(MesosExternalShuffleClient.class);
private final ScheduledExecutorService heartbeaterThread =
Executors.newSingleThreadScheduledExecutor(

View file

@ -58,7 +58,7 @@ import org.apache.spark.network.yarn.util.HadoopConfigProvider;
* the service's.
*/
public class YarnShuffleService extends AuxiliaryService {
private final Logger logger = LoggerFactory.getLogger(YarnShuffleService.class);
private static final Logger logger = LoggerFactory.getLogger(YarnShuffleService.class);
// Port on which the shuffle server listens for fetch requests
private static final String SPARK_SHUFFLE_SERVICE_PORT_KEY = "spark.shuffle.service.port";

View file

@ -53,7 +53,7 @@ import org.apache.spark.util.Utils;
*/
public class TaskMemoryManager {
private final Logger logger = LoggerFactory.getLogger(TaskMemoryManager.class);
private static final Logger logger = LoggerFactory.getLogger(TaskMemoryManager.class);
/** The number of bits used to address the page table. */
private static final int PAGE_NUMBER_BITS = 13;

View file

@ -73,7 +73,7 @@ import org.apache.spark.util.Utils;
*/
final class BypassMergeSortShuffleWriter<K, V> extends ShuffleWriter<K, V> {
private final Logger logger = LoggerFactory.getLogger(BypassMergeSortShuffleWriter.class);
private static final Logger logger = LoggerFactory.getLogger(BypassMergeSortShuffleWriter.class);
private final int fileBufferSize;
private final boolean transferToEnabled;

View file

@ -62,7 +62,7 @@ import org.apache.spark.util.Utils;
*/
final class ShuffleExternalSorter extends MemoryConsumer {
private final Logger logger = LoggerFactory.getLogger(ShuffleExternalSorter.class);
private static final Logger logger = LoggerFactory.getLogger(ShuffleExternalSorter.class);
@VisibleForTesting
static final int DISK_WRITE_BUFFER_SIZE = 1024 * 1024;

View file

@ -56,7 +56,7 @@ import org.apache.spark.util.Utils;
@Private
public class UnsafeShuffleWriter<K, V> extends ShuffleWriter<K, V> {
private final Logger logger = LoggerFactory.getLogger(UnsafeShuffleWriter.class);
private static final Logger logger = LoggerFactory.getLogger(UnsafeShuffleWriter.class);
private static final ClassTag<Object> OBJECT_CLASS_TAG = ClassTag$.MODULE$.Object();

View file

@ -64,7 +64,7 @@ import org.apache.spark.util.collection.unsafe.sort.UnsafeSorterSpillWriter;
*/
public final class BytesToBytesMap extends MemoryConsumer {
private final Logger logger = LoggerFactory.getLogger(BytesToBytesMap.class);
private static final Logger logger = LoggerFactory.getLogger(BytesToBytesMap.class);
private static final HashMapGrowthStrategy growthStrategy = HashMapGrowthStrategy.DOUBLING;

View file

@ -44,7 +44,7 @@ import org.apache.spark.util.Utils;
*/
public final class UnsafeExternalSorter extends MemoryConsumer {
private final Logger logger = LoggerFactory.getLogger(UnsafeExternalSorter.class);
private static final Logger logger = LoggerFactory.getLogger(UnsafeExternalSorter.class);
@Nullable
private final PrefixComparator prefixComparator;