Skip to content

Commit

Permalink
[SPARK-48291][CORE] Rename Java Logger as SparkLogger
Browse files Browse the repository at this point in the history
### What changes were proposed in this pull request?

Two new classes `org.apache.spark.internal.Logger` and `org.apache.spark.internal.LoggerFactory` were introduced from #46301.
Given that Logger is a widely recognized **interface** in Log4j, it may lead to confusion to have a class with the same name. To avoid this and clarify its purpose within the Spark framework, I propose renaming `org.apache.spark.internal.Logger` to `org.apache.spark.internal.SparkLogger`. Similarly, to maintain consistency, `org.apache.spark.internal.LoggerFactory` should be renamed to `org.apache.spark.internal.SparkLoggerFactory`.

### Why are the changes needed?

To avoid naming confusion and clarify the java Spark logger purpose within the logging framework

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

GA tests
### Was this patch authored or co-authored using generative AI tooling?

No

Closes #46600 from gengliangwang/refactorLogger.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
  • Loading branch information
gengliangwang committed May 15, 2024
1 parent a2d93d1 commit a252cbd
Show file tree
Hide file tree
Showing 74 changed files with 255 additions and 231 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -35,8 +35,8 @@
import io.netty.handler.timeout.IdleStateHandler;
import io.netty.handler.codec.MessageToMessageEncoder;

import org.apache.spark.internal.Logger;
import org.apache.spark.internal.LoggerFactory;
import org.apache.spark.internal.SparkLogger;
import org.apache.spark.internal.SparkLoggerFactory;
import org.apache.spark.network.client.TransportClient;
import org.apache.spark.network.client.TransportClientBootstrap;
import org.apache.spark.network.client.TransportClientFactory;
Expand Down Expand Up @@ -73,7 +73,7 @@
* processes to send messages back to the client on an existing channel.
*/
public class TransportContext implements Closeable {
private static final Logger logger = LoggerFactory.getLogger(TransportContext.class);
private static final SparkLogger logger = SparkLoggerFactory.getLogger(TransportContext.class);

private static final NettyLogger nettyLogger = new NettyLogger();
private final TransportConf conf;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,8 +36,8 @@
import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle;

import org.apache.spark.internal.Logger;
import org.apache.spark.internal.LoggerFactory;
import org.apache.spark.internal.SparkLogger;
import org.apache.spark.internal.SparkLoggerFactory;
import org.apache.spark.internal.LogKeys;
import org.apache.spark.internal.MDC;
import org.apache.spark.network.buffer.ManagedBuffer;
Expand Down Expand Up @@ -73,7 +73,7 @@
* Concurrency: thread safe and can be called from multiple threads.
*/
public class TransportClient implements Closeable {
private static final Logger logger = LoggerFactory.getLogger(TransportClient.class);
private static final SparkLogger logger = SparkLoggerFactory.getLogger(TransportClient.class);

private final Channel channel;
private final TransportResponseHandler handler;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,8 +43,8 @@
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.GenericFutureListener;

import org.apache.spark.internal.Logger;
import org.apache.spark.internal.LoggerFactory;
import org.apache.spark.internal.SparkLogger;
import org.apache.spark.internal.SparkLoggerFactory;
import org.apache.spark.internal.LogKeys;
import org.apache.spark.internal.MDC;
import org.apache.spark.network.TransportContext;
Expand Down Expand Up @@ -79,7 +79,8 @@ private static class ClientPool {
}
}

private static final Logger logger = LoggerFactory.getLogger(TransportClientFactory.class);
private static final SparkLogger logger =
SparkLoggerFactory.getLogger(TransportClientFactory.class);

private final TransportContext context;
private final TransportConf conf;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,8 +29,8 @@
import org.apache.commons.lang3.tuple.ImmutablePair;
import org.apache.commons.lang3.tuple.Pair;

import org.apache.spark.internal.Logger;
import org.apache.spark.internal.LoggerFactory;
import org.apache.spark.internal.SparkLogger;
import org.apache.spark.internal.SparkLoggerFactory;
import org.apache.spark.internal.LogKeys;
import org.apache.spark.internal.MDC;
import org.apache.spark.network.protocol.ChunkFetchFailure;
Expand All @@ -53,7 +53,8 @@
* Concurrency: thread safe and can be called from multiple threads.
*/
public class TransportResponseHandler extends MessageHandler<ResponseMessage> {
private static final Logger logger = LoggerFactory.getLogger(TransportResponseHandler.class);
private static final SparkLogger logger =
SparkLoggerFactory.getLogger(TransportResponseHandler.class);

private final Channel channel;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,8 +27,8 @@
import io.netty.buffer.Unpooled;
import io.netty.channel.Channel;

import org.apache.spark.internal.Logger;
import org.apache.spark.internal.LoggerFactory;
import org.apache.spark.internal.SparkLogger;
import org.apache.spark.internal.SparkLoggerFactory;
import org.apache.spark.network.client.TransportClient;
import org.apache.spark.network.client.TransportClientBootstrap;
import org.apache.spark.network.sasl.SaslClientBootstrap;
Expand All @@ -47,7 +47,7 @@
*/
public class AuthClientBootstrap implements TransportClientBootstrap {

private static final Logger LOG = LoggerFactory.getLogger(AuthClientBootstrap.class);
private static final SparkLogger LOG = SparkLoggerFactory.getLogger(AuthClientBootstrap.class);

private final TransportConf conf;
private final String appId;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,8 @@
import io.netty.buffer.Unpooled;
import io.netty.channel.Channel;

import org.apache.spark.internal.Logger;
import org.apache.spark.internal.LoggerFactory;
import org.apache.spark.internal.SparkLogger;
import org.apache.spark.internal.SparkLoggerFactory;
import org.apache.spark.internal.LogKeys;
import org.apache.spark.internal.MDC;
import org.apache.spark.network.client.RpcResponseCallback;
Expand All @@ -48,7 +48,7 @@
* authenticated. A connection may be authenticated at most once.
*/
class AuthRpcHandler extends AbstractAuthRpcHandler {
private static final Logger LOG = LoggerFactory.getLogger(AuthRpcHandler.class);
private static final SparkLogger LOG = SparkLoggerFactory.getLogger(AuthRpcHandler.class);

/** Transport configuration. */
private final TransportConf conf;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,8 +24,8 @@
import io.netty.channel.ChannelHandlerContext;
import io.netty.handler.codec.MessageToMessageDecoder;

import org.apache.spark.internal.Logger;
import org.apache.spark.internal.LoggerFactory;
import org.apache.spark.internal.SparkLogger;
import org.apache.spark.internal.SparkLoggerFactory;

/**
* Decoder used by the client side to encode server-to-client responses.
Expand All @@ -34,7 +34,7 @@
@ChannelHandler.Sharable
public final class MessageDecoder extends MessageToMessageDecoder<ByteBuf> {

private static final Logger logger = LoggerFactory.getLogger(MessageDecoder.class);
private static final SparkLogger logger = SparkLoggerFactory.getLogger(MessageDecoder.class);

public static final MessageDecoder INSTANCE = new MessageDecoder();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,8 +25,8 @@
import io.netty.handler.codec.MessageToMessageEncoder;

import org.apache.spark.internal.LogKeys;
import org.apache.spark.internal.Logger;
import org.apache.spark.internal.LoggerFactory;
import org.apache.spark.internal.SparkLogger;
import org.apache.spark.internal.SparkLoggerFactory;
import org.apache.spark.internal.MDC;

/**
Expand All @@ -36,7 +36,7 @@
@ChannelHandler.Sharable
public final class MessageEncoder extends MessageToMessageEncoder<Message> {

private static final Logger logger = LoggerFactory.getLogger(MessageEncoder.class);
private static final SparkLogger logger = SparkLoggerFactory.getLogger(MessageEncoder.class);

public static final MessageEncoder INSTANCE = new MessageEncoder();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,8 @@
import io.netty.handler.codec.MessageToMessageEncoder;
import io.netty.handler.stream.ChunkedStream;

import org.apache.spark.internal.Logger;
import org.apache.spark.internal.LoggerFactory;
import org.apache.spark.internal.SparkLogger;
import org.apache.spark.internal.SparkLoggerFactory;
import org.apache.spark.internal.LogKeys;
import org.apache.spark.internal.MDC;

Expand All @@ -38,7 +38,7 @@
@ChannelHandler.Sharable
public final class SslMessageEncoder extends MessageToMessageEncoder<Message> {

private static final Logger logger = LoggerFactory.getLogger(SslMessageEncoder.class);
private static final SparkLogger logger = SparkLoggerFactory.getLogger(SslMessageEncoder.class);

private SslMessageEncoder() {}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,8 +27,8 @@
import io.netty.buffer.Unpooled;
import io.netty.channel.Channel;

import org.apache.spark.internal.Logger;
import org.apache.spark.internal.LoggerFactory;
import org.apache.spark.internal.SparkLogger;
import org.apache.spark.internal.SparkLoggerFactory;
import org.apache.spark.network.client.TransportClient;
import org.apache.spark.network.client.TransportClientBootstrap;
import org.apache.spark.network.util.JavaUtils;
Expand All @@ -39,7 +39,7 @@
* server should be setup with a {@link SaslRpcHandler} with matching keys for the given appId.
*/
public class SaslClientBootstrap implements TransportClientBootstrap {
private static final Logger logger = LoggerFactory.getLogger(SaslClientBootstrap.class);
private static final SparkLogger logger = SparkLoggerFactory.getLogger(SaslClientBootstrap.class);

private final TransportConf conf;
private final String appId;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,8 +25,8 @@
import io.netty.buffer.Unpooled;
import io.netty.channel.Channel;

import org.apache.spark.internal.Logger;
import org.apache.spark.internal.LoggerFactory;
import org.apache.spark.internal.SparkLogger;
import org.apache.spark.internal.SparkLoggerFactory;
import org.apache.spark.network.client.RpcResponseCallback;
import org.apache.spark.network.client.TransportClient;
import org.apache.spark.network.server.AbstractAuthRpcHandler;
Expand All @@ -43,7 +43,7 @@
* which are individual RPCs.
*/
public class SaslRpcHandler extends AbstractAuthRpcHandler {
private static final Logger logger = LoggerFactory.getLogger(SaslRpcHandler.class);
private static final SparkLogger logger = SparkLoggerFactory.getLogger(SaslRpcHandler.class);

/** Transport configuration. */
private final TransportConf conf;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,8 +32,8 @@
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;

import org.apache.spark.internal.Logger;
import org.apache.spark.internal.LoggerFactory;
import org.apache.spark.internal.SparkLogger;
import org.apache.spark.internal.SparkLoggerFactory;

import static org.apache.spark.network.sasl.SparkSaslServer.*;

Expand All @@ -43,7 +43,7 @@
* firstToken, which is then followed by a set of challenges and responses.
*/
public class SparkSaslClient implements SaslEncryptionBackend {
private static final Logger logger = LoggerFactory.getLogger(SparkSaslClient.class);
private static final SparkLogger logger = SparkLoggerFactory.getLogger(SparkSaslClient.class);

private final String secretKeyId;
private final SecretKeyHolder secretKeyHolder;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,16 +37,16 @@
import io.netty.buffer.Unpooled;
import io.netty.handler.codec.base64.Base64;

import org.apache.spark.internal.Logger;
import org.apache.spark.internal.LoggerFactory;
import org.apache.spark.internal.SparkLogger;
import org.apache.spark.internal.SparkLoggerFactory;

/**
* A SASL Server for Spark which simply keeps track of the state of a single SASL session, from the
* initial state to the "authenticated" state. (It is not a server in the sense of accepting
* connections on some socket.)
*/
public class SparkSaslServer implements SaslEncryptionBackend {
private static final Logger logger = LoggerFactory.getLogger(SparkSaslServer.class);
private static final SparkLogger logger = SparkLoggerFactory.getLogger(SparkSaslServer.class);

/**
* This is passed as the server name when creating the sasl client/server.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,8 +27,8 @@
import io.netty.channel.SimpleChannelInboundHandler;

import org.apache.spark.internal.LogKeys;
import org.apache.spark.internal.Logger;
import org.apache.spark.internal.LoggerFactory;
import org.apache.spark.internal.SparkLogger;
import org.apache.spark.internal.SparkLoggerFactory;
import org.apache.spark.internal.MDC;
import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.client.TransportClient;
Expand All @@ -51,7 +51,8 @@
* registering executors, or waiting for response for an OpenBlocks messages.
*/
public class ChunkFetchRequestHandler extends SimpleChannelInboundHandler<ChunkFetchRequest> {
private static final Logger logger = LoggerFactory.getLogger(ChunkFetchRequestHandler.class);
private static final SparkLogger logger =
SparkLoggerFactory.getLogger(ChunkFetchRequestHandler.class);

private final TransportClient client;
private final StreamManager streamManager;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,8 +29,8 @@
import org.apache.commons.lang3.tuple.ImmutablePair;
import org.apache.commons.lang3.tuple.Pair;

import org.apache.spark.internal.Logger;
import org.apache.spark.internal.LoggerFactory;
import org.apache.spark.internal.SparkLogger;
import org.apache.spark.internal.SparkLoggerFactory;
import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.client.TransportClient;

Expand All @@ -39,7 +39,8 @@
* individually fetched as chunks by the client. Each registered buffer is one chunk.
*/
public class OneForOneStreamManager extends StreamManager {
private static final Logger logger = LoggerFactory.getLogger(OneForOneStreamManager.class);
private static final SparkLogger logger =
SparkLoggerFactory.getLogger(OneForOneStreamManager.class);

private final AtomicLong nextStreamId;
private final ConcurrentHashMap<Long, StreamState> streams;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,8 @@

import java.nio.ByteBuffer;

import org.apache.spark.internal.Logger;
import org.apache.spark.internal.LoggerFactory;
import org.apache.spark.internal.SparkLogger;
import org.apache.spark.internal.SparkLoggerFactory;
import org.apache.spark.network.client.MergedBlockMetaResponseCallback;
import org.apache.spark.network.client.RpcResponseCallback;
import org.apache.spark.network.client.StreamCallbackWithID;
Expand Down Expand Up @@ -122,7 +122,7 @@ public void exceptionCaught(Throwable cause, TransportClient client) { }

private static class OneWayRpcCallback implements RpcResponseCallback {

private static final Logger logger = LoggerFactory.getLogger(OneWayRpcCallback.class);
private static final SparkLogger logger = SparkLoggerFactory.getLogger(OneWayRpcCallback.class);

@Override
public void onSuccess(ByteBuffer response) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,8 +23,8 @@
import io.netty.handler.timeout.IdleStateEvent;
import org.apache.spark.network.TransportContext;

import org.apache.spark.internal.Logger;
import org.apache.spark.internal.LoggerFactory;
import org.apache.spark.internal.SparkLogger;
import org.apache.spark.internal.SparkLoggerFactory;
import org.apache.spark.internal.LogKeys;
import org.apache.spark.internal.MDC;
import org.apache.spark.network.client.TransportClient;
Expand Down Expand Up @@ -53,7 +53,8 @@
* timeout if the client is continuously sending but getting no responses, for simplicity.
*/
public class TransportChannelHandler extends SimpleChannelInboundHandler<Message> {
private static final Logger logger = LoggerFactory.getLogger(TransportChannelHandler.class);
private static final SparkLogger logger =
SparkLoggerFactory.getLogger(TransportChannelHandler.class);

private final TransportClient client;
private final TransportResponseHandler responseHandler;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,8 +25,8 @@
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;

import org.apache.spark.internal.Logger;
import org.apache.spark.internal.LoggerFactory;
import org.apache.spark.internal.SparkLogger;
import org.apache.spark.internal.SparkLoggerFactory;
import org.apache.spark.internal.LogKeys;
import org.apache.spark.internal.MDC;
import org.apache.spark.network.buffer.ManagedBuffer;
Expand All @@ -46,7 +46,8 @@
*/
public class TransportRequestHandler extends MessageHandler<RequestMessage> {

private static final Logger logger = LoggerFactory.getLogger(TransportRequestHandler.class);
private static final SparkLogger logger =
SparkLoggerFactory.getLogger(TransportRequestHandler.class);

/** The Netty channel that this handler is associated with. */
private final Channel channel;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,16 +35,16 @@
import io.netty.channel.socket.SocketChannel;
import org.apache.commons.lang3.SystemUtils;

import org.apache.spark.internal.Logger;
import org.apache.spark.internal.LoggerFactory;
import org.apache.spark.internal.SparkLogger;
import org.apache.spark.internal.SparkLoggerFactory;
import org.apache.spark.network.TransportContext;
import org.apache.spark.network.util.*;

/**
* Server for the efficient, low-level streaming service.
*/
public class TransportServer implements Closeable {
private static final Logger logger = LoggerFactory.getLogger(TransportServer.class);
private static final SparkLogger logger = SparkLoggerFactory.getLogger(TransportServer.class);

private final TransportContext context;
private final TransportConf conf;
Expand Down
Loading

0 comments on commit a252cbd

Please sign in to comment.