diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java index 08449a2284404..071c239e7bf1b 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java @@ -364,8 +364,8 @@ public void operationComplete(Future future) throws Exception { } } else { logger.error("Failed to send RPC {} to {}", future.cause(), - MDC.of(LogKeys.REQUEST_ID$.MODULE$, requestId), - MDC.of(LogKeys.HOST_PORT$.MODULE$, getRemoteAddress(channel))); + MDC.of(LogKeys.REQUEST_ID, requestId), + MDC.of(LogKeys.HOST_PORT, getRemoteAddress(channel))); channel.close(); try { String errorMsg = String.format("Failed to send RPC %s to %s: %s", requestId, diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java index d64b8c8f838e9..74111a22e72d0 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java @@ -193,9 +193,9 @@ public TransportClient createClient(String remoteHost, int remotePort, boolean f final String resolvMsg = resolvedAddress.isUnresolved() ? "failed" : "succeed"; if (hostResolveTimeMs > 2000) { logger.warn("DNS resolution {} for {} took {} ms", - MDC.of(LogKeys.STATUS$.MODULE$, resolvMsg), - MDC.of(LogKeys.HOST_PORT$.MODULE$, resolvedAddress), - MDC.of(LogKeys.TIME$.MODULE$, hostResolveTimeMs)); + MDC.of(LogKeys.STATUS, resolvMsg), + MDC.of(LogKeys.HOST_PORT, resolvedAddress), + MDC.of(LogKeys.TIME, hostResolveTimeMs)); } else { logger.trace("DNS resolution {} for {} took {} ms", resolvMsg, resolvedAddress, hostResolveTimeMs); @@ -210,7 +210,7 @@ public TransportClient createClient(String remoteHost, int remotePort, boolean f return cachedClient; } else { logger.info("Found inactive connection to {}, creating a new one.", - MDC.of(LogKeys.HOST_PORT$.MODULE$, resolvedAddress)); + MDC.of(LogKeys.HOST_PORT, resolvedAddress)); } } // If this connection should fast fail when last connection failed in last fast fail time @@ -314,7 +314,7 @@ public void operationComplete(final Future handshakeFuture) { logger.debug("{} successfully completed TLS handshake to ", address); } else { logger.info("failed to complete TLS handshake to {}", handshakeFuture.cause(), - MDC.of(LogKeys.HOST_PORT$.MODULE$, address)); + MDC.of(LogKeys.HOST_PORT, address)); cf.channel().close(); } } @@ -340,7 +340,7 @@ public void operationComplete(final Future handshakeFuture) { } catch (Exception e) { // catch non-RuntimeExceptions too as bootstrap may be written in Scala long bootstrapTimeMs = (System.nanoTime() - preBootstrap) / 1000000; logger.error("Exception while bootstrapping client after {} ms", e, - MDC.of(LogKeys.BOOTSTRAP_TIME$.MODULE$, bootstrapTimeMs)); + MDC.of(LogKeys.BOOTSTRAP_TIME, bootstrapTimeMs)); client.close(); Throwables.throwIfUnchecked(e); throw new RuntimeException(e); @@ -348,9 +348,9 @@ public void operationComplete(final Future handshakeFuture) { long postBootstrap = System.nanoTime(); logger.info("Successfully created connection to {} after {} ms ({} ms spent in bootstraps)", - MDC.of(LogKeys.HOST_PORT$.MODULE$, address), - MDC.of(LogKeys.ELAPSED_TIME$.MODULE$, (postBootstrap - preConnect) / 1000000), - MDC.of(LogKeys.BOOTSTRAP_TIME$.MODULE$, (postBootstrap - preBootstrap) / 1000000)); + MDC.of(LogKeys.HOST_PORT, address), + MDC.of(LogKeys.ELAPSED_TIME, (postBootstrap - preConnect) / 1000000), + MDC.of(LogKeys.BOOTSTRAP_TIME, (postBootstrap - preBootstrap) / 1000000)); return client; } diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java index f387c30438eeb..d27fa08d829bb 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java @@ -145,8 +145,8 @@ public void channelInactive() { if (hasOutstandingRequests()) { String remoteAddress = getRemoteAddress(channel); logger.error("Still have {} requests outstanding when connection from {} is closed", - MDC.of(LogKeys.COUNT$.MODULE$, numOutstandingRequests()), - MDC.of(LogKeys.HOST_PORT$.MODULE$, remoteAddress)); + MDC.of(LogKeys.COUNT, numOutstandingRequests()), + MDC.of(LogKeys.HOST_PORT, remoteAddress)); failOutstandingRequests(new IOException("Connection from " + remoteAddress + " closed")); } } @@ -156,8 +156,8 @@ public void exceptionCaught(Throwable cause) { if (hasOutstandingRequests()) { String remoteAddress = getRemoteAddress(channel); logger.error("Still have {} requests outstanding when connection from {} is closed", - MDC.of(LogKeys.COUNT$.MODULE$, numOutstandingRequests()), - MDC.of(LogKeys.HOST_PORT$.MODULE$, remoteAddress)); + MDC.of(LogKeys.COUNT, numOutstandingRequests()), + MDC.of(LogKeys.HOST_PORT, remoteAddress)); failOutstandingRequests(cause); } } @@ -168,8 +168,8 @@ public void handle(ResponseMessage message) throws Exception { ChunkReceivedCallback listener = outstandingFetches.get(resp.streamChunkId); if (listener == null) { logger.warn("Ignoring response for block {} from {} since it is not outstanding", - MDC.of(LogKeys.STREAM_CHUNK_ID$.MODULE$, resp.streamChunkId), - MDC.of(LogKeys.HOST_PORT$.MODULE$, getRemoteAddress(channel))); + MDC.of(LogKeys.STREAM_CHUNK_ID, resp.streamChunkId), + MDC.of(LogKeys.HOST_PORT, getRemoteAddress(channel))); resp.body().release(); } else { outstandingFetches.remove(resp.streamChunkId); @@ -180,9 +180,9 @@ public void handle(ResponseMessage message) throws Exception { ChunkReceivedCallback listener = outstandingFetches.get(resp.streamChunkId); if (listener == null) { logger.warn("Ignoring response for block {} from {} ({}) since it is not outstanding", - MDC.of(LogKeys.STREAM_CHUNK_ID$.MODULE$, resp.streamChunkId), - MDC.of(LogKeys.HOST_PORT$.MODULE$, getRemoteAddress(channel)), - MDC.of(LogKeys.ERROR$.MODULE$, resp.errorString)); + MDC.of(LogKeys.STREAM_CHUNK_ID, resp.streamChunkId), + MDC.of(LogKeys.HOST_PORT, getRemoteAddress(channel)), + MDC.of(LogKeys.ERROR, resp.errorString)); } else { outstandingFetches.remove(resp.streamChunkId); listener.onFailure(resp.streamChunkId.chunkIndex(), new ChunkFetchFailureException( @@ -192,9 +192,9 @@ public void handle(ResponseMessage message) throws Exception { RpcResponseCallback listener = (RpcResponseCallback) outstandingRpcs.get(resp.requestId); if (listener == null) { logger.warn("Ignoring response for RPC {} from {} ({} bytes) since it is not outstanding", - MDC.of(LogKeys.REQUEST_ID$.MODULE$, resp.requestId), - MDC.of(LogKeys.HOST_PORT$.MODULE$, getRemoteAddress(channel)), - MDC.of(LogKeys.RESPONSE_BODY_SIZE$.MODULE$, resp.body().size())); + MDC.of(LogKeys.REQUEST_ID, resp.requestId), + MDC.of(LogKeys.HOST_PORT, getRemoteAddress(channel)), + MDC.of(LogKeys.RESPONSE_BODY_SIZE, resp.body().size())); resp.body().release(); } else { outstandingRpcs.remove(resp.requestId); @@ -208,9 +208,9 @@ public void handle(ResponseMessage message) throws Exception { BaseResponseCallback listener = outstandingRpcs.get(resp.requestId); if (listener == null) { logger.warn("Ignoring response for RPC {} from {} ({}) since it is not outstanding", - MDC.of(LogKeys.REQUEST_ID$.MODULE$, resp.requestId), - MDC.of(LogKeys.HOST_PORT$.MODULE$, getRemoteAddress(channel)), - MDC.of(LogKeys.ERROR$.MODULE$, resp.errorString)); + MDC.of(LogKeys.REQUEST_ID, resp.requestId), + MDC.of(LogKeys.HOST_PORT, getRemoteAddress(channel)), + MDC.of(LogKeys.ERROR, resp.errorString)); } else { outstandingRpcs.remove(resp.requestId); listener.onFailure(new RuntimeException(resp.errorString)); @@ -222,9 +222,9 @@ public void handle(ResponseMessage message) throws Exception { if (listener == null) { logger.warn("Ignoring response for MergedBlockMetaRequest {} from {} ({} bytes) since " + "it is not outstanding", - MDC.of(LogKeys.REQUEST_ID$.MODULE$, resp.requestId), - MDC.of(LogKeys.HOST_PORT$.MODULE$, getRemoteAddress(channel)), - MDC.of(LogKeys.RESPONSE_BODY_SIZE$.MODULE$, resp.body().size())); + MDC.of(LogKeys.REQUEST_ID, resp.requestId), + MDC.of(LogKeys.HOST_PORT, getRemoteAddress(channel)), + MDC.of(LogKeys.RESPONSE_BODY_SIZE, resp.body().size())); } else { outstandingRpcs.remove(resp.requestId); listener.onSuccess(resp.getNumChunks(), resp.body()); @@ -269,7 +269,7 @@ public void handle(ResponseMessage message) throws Exception { } } else { logger.warn("Stream failure with unknown callback: {}", - MDC.of(LogKeys.ERROR$.MODULE$, resp.error)); + MDC.of(LogKeys.ERROR, resp.error)); } } else { throw new IllegalStateException("Unknown response type: " + message.type()); diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java index 087e3d21e22bb..2a6e36ea45e38 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java @@ -93,7 +93,7 @@ protected boolean doAuthChallenge( } catch (RuntimeException e) { if (conf.saslFallback()) { LOG.warn("Failed to parse new auth challenge, reverting to SASL for client {}.", - MDC.of(LogKeys.HOST_PORT$.MODULE$, channel.remoteAddress())); + MDC.of(LogKeys.HOST_PORT, channel.remoteAddress())); saslHandler = new SaslRpcHandler(conf, channel, null, secretKeyHolder); message.position(position); message.limit(limit); diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java index ab20fb908eb42..3f23c17939e6c 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java @@ -66,8 +66,8 @@ public void encode(ChannelHandlerContext ctx, Message in, List out) thro // Re-encode this message as a failure response. String error = e.getMessage() != null ? e.getMessage() : "null"; logger.error("Error processing {} for client {}", e, - MDC.of(LogKeys.MESSAGE$.MODULE$, in), - MDC.of(LogKeys.HOST_PORT$.MODULE$, ctx.channel().remoteAddress())); + MDC.of(LogKeys.MESSAGE, in), + MDC.of(LogKeys.HOST_PORT, ctx.channel().remoteAddress())); encode(ctx, resp.createFailureResponse(error), out); } else { throw e; diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/SslMessageEncoder.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/SslMessageEncoder.java index abe6ccca7bfd6..083e45b3d6bdb 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/SslMessageEncoder.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/SslMessageEncoder.java @@ -71,8 +71,8 @@ public void encode(ChannelHandlerContext ctx, Message in, List out) thro // Re-encode this message as a failure response. String error = e.getMessage() != null ? e.getMessage() : "null"; logger.error("Error processing {} for client {}", e, - MDC.of(LogKeys.MESSAGE$.MODULE$, in), - MDC.of(LogKeys.HOST_PORT$.MODULE$, ctx.channel().remoteAddress())); + MDC.of(LogKeys.MESSAGE, in), + MDC.of(LogKeys.HOST_PORT, ctx.channel().remoteAddress())); encode(ctx, resp.createFailureResponse(error), out); } else { throw e; diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/ChunkFetchRequestHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/ChunkFetchRequestHandler.java index cc0bed7ed5b6d..11641ddacd58b 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/ChunkFetchRequestHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/ChunkFetchRequestHandler.java @@ -74,7 +74,7 @@ public ChunkFetchRequestHandler( @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { logger.warn("Exception in connection from {}", cause, - MDC.of(LogKeys.HOST_PORT$.MODULE$, getRemoteAddress(ctx.channel()))); + MDC.of(LogKeys.HOST_PORT, getRemoteAddress(ctx.channel()))); ctx.close(); } @@ -96,8 +96,8 @@ public void processFetchRequest( long chunksBeingTransferred = streamManager.chunksBeingTransferred(); if (chunksBeingTransferred >= maxChunksBeingTransferred) { logger.warn("The number of chunks being transferred {} is above {}, close the connection.", - MDC.of(LogKeys.NUM_CHUNKS$.MODULE$, chunksBeingTransferred), - MDC.of(LogKeys.MAX_NUM_CHUNKS$.MODULE$, maxChunksBeingTransferred)); + MDC.of(LogKeys.NUM_CHUNKS, chunksBeingTransferred), + MDC.of(LogKeys.MAX_NUM_CHUNKS, maxChunksBeingTransferred)); channel.close(); return; } @@ -111,8 +111,8 @@ public void processFetchRequest( } } catch (Exception e) { logger.error("Error opening block {} for request from {}", e, - MDC.of(LogKeys.STREAM_CHUNK_ID$.MODULE$, msg.streamChunkId), - MDC.of(LogKeys.HOST_PORT$.MODULE$, getRemoteAddress(channel))); + MDC.of(LogKeys.STREAM_CHUNK_ID, msg.streamChunkId), + MDC.of(LogKeys.HOST_PORT, getRemoteAddress(channel))); respond(channel, new ChunkFetchFailure(msg.streamChunkId, Throwables.getStackTraceAsString(e))); return; @@ -153,8 +153,8 @@ private ChannelFuture respond( } else { logger.error("Error sending result {} to {}; closing connection", future.cause(), - MDC.of(LogKeys.RESULT$.MODULE$, result), - MDC.of(LogKeys.HOST_PORT$.MODULE$, remoteAddress)); + MDC.of(LogKeys.RESULT, result), + MDC.of(LogKeys.HOST_PORT, remoteAddress)); channel.close(); } }); diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java index 283f0f0a431fd..d0df24873cbce 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java @@ -88,7 +88,7 @@ public TransportClient getClient() { @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { logger.warn("Exception in connection from {}", cause, - MDC.of(LogKeys.HOST_PORT$.MODULE$, getRemoteAddress(ctx.channel()))); + MDC.of(LogKeys.HOST_PORT, getRemoteAddress(ctx.channel()))); requestHandler.exceptionCaught(cause); responseHandler.exceptionCaught(cause); ctx.close(); @@ -168,9 +168,9 @@ public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exc logger.error("Connection to {} has been quiet for {} ms while there are outstanding " + "requests. Assuming connection is dead; please adjust" + " spark.{}.io.connectionTimeout if this is wrong.", - MDC.of(LogKeys.HOST_PORT$.MODULE$, address), - MDC.of(LogKeys.TIMEOUT$.MODULE$, requestTimeoutNs / 1000 / 1000), - MDC.of(LogKeys.MODULE_NAME$.MODULE$, transportContext.getConf().getModuleName())); + MDC.of(LogKeys.HOST_PORT, address), + MDC.of(LogKeys.TIMEOUT, requestTimeoutNs / 1000 / 1000), + MDC.of(LogKeys.MODULE_NAME, transportContext.getConf().getModuleName())); client.timeOut(); ctx.close(); } else if (closeIdleConnections) { diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java index 2727051894b7a..37174a66c6d6d 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java @@ -132,8 +132,8 @@ private void processStreamRequest(final StreamRequest req) { long chunksBeingTransferred = streamManager.chunksBeingTransferred(); if (chunksBeingTransferred >= maxChunksBeingTransferred) { logger.warn("The number of chunks being transferred {} is above {}, close the connection.", - MDC.of(LogKeys.NUM_CHUNKS$.MODULE$, chunksBeingTransferred), - MDC.of(LogKeys.MAX_NUM_CHUNKS$.MODULE$, maxChunksBeingTransferred)); + MDC.of(LogKeys.NUM_CHUNKS, chunksBeingTransferred), + MDC.of(LogKeys.MAX_NUM_CHUNKS, maxChunksBeingTransferred)); channel.close(); return; } @@ -143,8 +143,8 @@ private void processStreamRequest(final StreamRequest req) { buf = streamManager.openStream(req.streamId); } catch (Exception e) { logger.error("Error opening stream {} for request from {}", e, - MDC.of(LogKeys.STREAM_ID$.MODULE$, req.streamId), - MDC.of(LogKeys.HOST_PORT$.MODULE$, getRemoteAddress(channel))); + MDC.of(LogKeys.STREAM_ID, req.streamId), + MDC.of(LogKeys.HOST_PORT, getRemoteAddress(channel))); respond(new StreamFailure(req.streamId, Throwables.getStackTraceAsString(e))); return; } @@ -177,8 +177,8 @@ public void onFailure(Throwable e) { }); } catch (Exception e) { logger.error("Error while invoking RpcHandler#receive() on RPC id {} from {}", e, - MDC.of(LogKeys.REQUEST_ID$.MODULE$, req.requestId), - MDC.of(LogKeys.HOST_PORT$.MODULE$, getRemoteAddress(channel))); + MDC.of(LogKeys.REQUEST_ID, req.requestId), + MDC.of(LogKeys.HOST_PORT, getRemoteAddress(channel))); respond(new RpcFailure(req.requestId, Throwables.getStackTraceAsString(e))); } finally { req.body().release(); @@ -264,8 +264,8 @@ public String getID() { new NioManagedBuffer(blockPushNonFatalFailure.getResponse()))); } else { logger.error("Error while invoking RpcHandler#receive() on RPC id {} from {}", e, - MDC.of(LogKeys.REQUEST_ID$.MODULE$, req.requestId), - MDC.of(LogKeys.HOST_PORT$.MODULE$, getRemoteAddress(channel))); + MDC.of(LogKeys.REQUEST_ID, req.requestId), + MDC.of(LogKeys.HOST_PORT, getRemoteAddress(channel))); respond(new RpcFailure(req.requestId, Throwables.getStackTraceAsString(e))); } // We choose to totally fail the channel, rather than trying to recover as we do in other @@ -282,7 +282,7 @@ private void processOneWayMessage(OneWayMessage req) { rpcHandler.receive(reverseClient, req.body().nioByteBuffer()); } catch (Exception e) { logger.error("Error while invoking RpcHandler#receive() for one-way message from {}.", e, - MDC.of(LogKeys.HOST_PORT$.MODULE$, getRemoteAddress(channel))); + MDC.of(LogKeys.HOST_PORT, getRemoteAddress(channel))); } finally { req.body().release(); } @@ -307,10 +307,10 @@ public void onFailure(Throwable e) { }); } catch (Exception e) { logger.error("Error while invoking receiveMergeBlockMetaReq() for appId {} shuffleId {} " - + "reduceId {} from {}", e, MDC.of(LogKeys.APP_ID$.MODULE$, req.appId), - MDC.of(LogKeys.SHUFFLE_ID$.MODULE$, req.shuffleId), - MDC.of(LogKeys.REDUCE_ID$.MODULE$, req.reduceId), - MDC.of(LogKeys.HOST_PORT$.MODULE$, getRemoteAddress(channel))); + + "reduceId {} from {}", e, MDC.of(LogKeys.APP_ID, req.appId), + MDC.of(LogKeys.SHUFFLE_ID, req.shuffleId), + MDC.of(LogKeys.REDUCE_ID, req.reduceId), + MDC.of(LogKeys.HOST_PORT, getRemoteAddress(channel))); respond(new RpcFailure(req.requestId, Throwables.getStackTraceAsString(e))); } } @@ -326,8 +326,8 @@ private ChannelFuture respond(Encodable result) { logger.trace("Sent result {} to client {}", result, remoteAddress); } else { logger.error("Error sending result {} to {}; closing connection", future.cause(), - MDC.of(LogKeys.RESULT$.MODULE$, result), - MDC.of(LogKeys.HOST_PORT$.MODULE$, remoteAddress)); + MDC.of(LogKeys.RESULT, result), + MDC.of(LogKeys.HOST_PORT, remoteAddress)); channel.close(); } }); diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/LevelDBProvider.java b/common/network-common/src/main/java/org/apache/spark/network/util/LevelDBProvider.java index 391931961a474..ec3e032102e4f 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/LevelDBProvider.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/LevelDBProvider.java @@ -50,7 +50,7 @@ public static DB initLevelDB(File dbFile, StoreVersion version, ObjectMapper map tmpDb = JniDBFactory.factory.open(dbFile, options); } catch (NativeDB.DBException e) { if (e.isNotFound() || e.getMessage().contains(" does not exist ")) { - logger.info("Creating state database at {}", MDC.of(LogKeys.PATH$.MODULE$, dbFile)); + logger.info("Creating state database at {}", MDC.of(LogKeys.PATH, dbFile)); options.createIfMissing(true); try { tmpDb = JniDBFactory.factory.open(dbFile, options); @@ -61,16 +61,16 @@ public static DB initLevelDB(File dbFile, StoreVersion version, ObjectMapper map // the leveldb file seems to be corrupt somehow. Lets just blow it away and create a new // one, so we can keep processing new apps logger.error("error opening leveldb file {}. Creating new file, will not be able to " + - "recover state for existing applications", e, MDC.of(LogKeys.PATH$.MODULE$, dbFile)); + "recover state for existing applications", e, MDC.of(LogKeys.PATH, dbFile)); if (dbFile.isDirectory()) { for (File f : dbFile.listFiles()) { if (!f.delete()) { - logger.warn("error deleting {}", MDC.of(LogKeys.PATH$.MODULE$, f.getPath())); + logger.warn("error deleting {}", MDC.of(LogKeys.PATH, f.getPath())); } } } if (!dbFile.delete()) { - logger.warn("error deleting {}", MDC.of(LogKeys.PATH$.MODULE$, dbFile.getPath())); + logger.warn("error deleting {}", MDC.of(LogKeys.PATH, dbFile.getPath())); } options.createIfMissing(true); try { diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/RocksDBProvider.java b/common/network-common/src/main/java/org/apache/spark/network/util/RocksDBProvider.java index 1753c124c9935..cea9207d3470a 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/RocksDBProvider.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/RocksDBProvider.java @@ -67,7 +67,7 @@ public static RocksDB initRockDB(File dbFile, StoreVersion version, ObjectMapper tmpDb = RocksDB.open(dbOptions, dbFile.toString()); } catch (RocksDBException e) { if (e.getStatus().getCode() == Status.Code.NotFound) { - logger.info("Creating state database at {}", MDC.of(LogKeys.PATH$.MODULE$, dbFile)); + logger.info("Creating state database at {}", MDC.of(LogKeys.PATH, dbFile)); dbOptions.setCreateIfMissing(true); try { tmpDb = RocksDB.open(dbOptions, dbFile.toString()); @@ -78,16 +78,16 @@ public static RocksDB initRockDB(File dbFile, StoreVersion version, ObjectMapper // the RocksDB file seems to be corrupt somehow. Let's just blow it away and create // a new one, so we can keep processing new apps logger.error("error opening rocksdb file {}. Creating new file, will not be able to " + - "recover state for existing applications", e, MDC.of(LogKeys.PATH$.MODULE$, dbFile)); + "recover state for existing applications", e, MDC.of(LogKeys.PATH, dbFile)); if (dbFile.isDirectory()) { for (File f : Objects.requireNonNull(dbFile.listFiles())) { if (!f.delete()) { - logger.warn("error deleting {}", MDC.of(LogKeys.PATH$.MODULE$, f.getPath())); + logger.warn("error deleting {}", MDC.of(LogKeys.PATH, f.getPath())); } } } if (!dbFile.delete()) { - logger.warn("error deleting {}", MDC.of(LogKeys.PATH$.MODULE$, dbFile.getPath())); + logger.warn("error deleting {}", MDC.of(LogKeys.PATH, dbFile.getPath())); } dbOptions.setCreateIfMissing(true); try { diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/sasl/ShuffleSecretManager.java b/common/network-shuffle/src/main/java/org/apache/spark/network/sasl/ShuffleSecretManager.java index d67f2a3099d35..625cb2e1257da 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/sasl/ShuffleSecretManager.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/sasl/ShuffleSecretManager.java @@ -55,7 +55,7 @@ public void registerApp(String appId, String shuffleSecret) { // to the applicationId since the secrets change between application attempts on yarn. shuffleSecretMap.put(appId, shuffleSecret); logger.info("Registered shuffle secret for application {}", - MDC.of(LogKeys.APP_ID$.MODULE$, appId)); + MDC.of(LogKeys.APP_ID, appId)); } /** @@ -72,7 +72,7 @@ public void registerApp(String appId, ByteBuffer shuffleSecret) { public void unregisterApp(String appId) { shuffleSecretMap.remove(appId); logger.info("Unregistered shuffle secret for application {}", - MDC.of(LogKeys.APP_ID$.MODULE$, appId)); + MDC.of(LogKeys.APP_ID, appId)); } /** diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java index dcb0a52b0d66c..ceb5d64699744 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java @@ -173,7 +173,7 @@ public void onSuccess(ByteBuffer response) { ((LocalDirsForExecutors) msgObj).getLocalDirsByExec()); } catch (Throwable t) { logger.warn("Error while trying to get the host local dirs for {}", t.getCause(), - MDC.of(LogKeys.EXECUTOR_IDS$.MODULE$, Arrays.toString(getLocalDirsMessage.execIds))); + MDC.of(LogKeys.EXECUTOR_IDS, Arrays.toString(getLocalDirsMessage.execIds))); hostLocalDirsCompletable.completeExceptionally(t); } } @@ -181,7 +181,7 @@ public void onSuccess(ByteBuffer response) { @Override public void onFailure(Throwable t) { logger.warn("Error while trying to get the host local dirs for {}", t.getCause(), - MDC.of(LogKeys.EXECUTOR_IDS$.MODULE$, Arrays.toString(getLocalDirsMessage.execIds))); + MDC.of(LogKeys.EXECUTOR_IDS, Arrays.toString(getLocalDirsMessage.execIds))); hostLocalDirsCompletable.completeExceptionally(t); } }); diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java index 5d33bfb345a9e..6958889995f1e 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java @@ -224,9 +224,9 @@ protected void handleMessage( } else if (msgObj instanceof RemoveShuffleMerge msg) { checkAuth(client, msg.appId); logger.info("Removing shuffle merge data for application {} shuffle {} shuffleMerge {}", - MDC.of(LogKeys.APP_ID$.MODULE$, msg.appId), - MDC.of(LogKeys.SHUFFLE_ID$.MODULE$, msg.shuffleId), - MDC.of(LogKeys.SHUFFLE_MERGE_ID$.MODULE$, msg.shuffleMergeId)); + MDC.of(LogKeys.APP_ID, msg.appId), + MDC.of(LogKeys.SHUFFLE_ID, msg.shuffleId), + MDC.of(LogKeys.SHUFFLE_MERGE_ID, msg.shuffleMergeId)); mergeManager.removeShuffleMerge(msg); } else if (msgObj instanceof DiagnoseCorruption msg) { checkAuth(client, msg.appId); diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java index 97723f77723d4..808bfc2726c36 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java @@ -106,7 +106,7 @@ private void setComparableAppAttemptId(String appAttemptId) { } catch (NumberFormatException e) { logger.warn("Push based shuffle requires comparable application attemptId, " + "but the appAttemptId {} cannot be parsed to Integer", e, - MDC.of(LogKeys.APP_ATTEMPT_ID$.MODULE$, appAttemptId)); + MDC.of(LogKeys.APP_ATTEMPT_ID, appAttemptId)); } } @@ -221,8 +221,8 @@ public void onFailure(Throwable e) { }); } catch (Exception e) { logger.error("Exception while sending finalizeShuffleMerge request to {}:{}", e, - MDC.of(LogKeys.HOST$.MODULE$, host), - MDC.of(LogKeys.PORT$.MODULE$, port)); + MDC.of(LogKeys.HOST, host), + MDC.of(LogKeys.PORT, port)); listener.onShuffleMergeFailure(e); } } @@ -322,8 +322,8 @@ public void onSuccess(ByteBuffer response) { } catch (Throwable t) { logger.warn("Error trying to remove blocks {} via external shuffle service from " + "executor: {}", t, - MDC.of(LogKeys.BLOCK_IDS$.MODULE$, Arrays.toString(blockIds)), - MDC.of(LogKeys.EXECUTOR_ID$.MODULE$, execId)); + MDC.of(LogKeys.BLOCK_IDS, Arrays.toString(blockIds)), + MDC.of(LogKeys.EXECUTOR_ID, execId)); numRemovedBlocksFuture.complete(0); } } @@ -331,8 +331,8 @@ public void onSuccess(ByteBuffer response) { @Override public void onFailure(Throwable e) { logger.warn("Error trying to remove blocks {} via external shuffle service from " + - "executor: {}", e, MDC.of(LogKeys.BLOCK_IDS$.MODULE$, Arrays.toString(blockIds)), - MDC.of(LogKeys.EXECUTOR_ID$.MODULE$, execId)); + "executor: {}", e, MDC.of(LogKeys.BLOCK_IDS, Arrays.toString(blockIds)), + MDC.of(LogKeys.EXECUTOR_ID, execId)); numRemovedBlocksFuture.complete(0); } }); diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 1950901fdcfaf..e30dc69df51c2 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -132,8 +132,8 @@ public ShuffleIndexInformation load(String filePath) throws IOException { db = DBProvider.initDB(dbBackend, this.registeredExecutorFile, CURRENT_VERSION, mapper); if (db != null) { logger.info("Use {} as the implementation of {}", - MDC.of(LogKeys.SHUFFLE_DB_BACKEND_NAME$.MODULE$, dbBackend), - MDC.of(LogKeys.SHUFFLE_DB_BACKEND_KEY$.MODULE$, Constants.SHUFFLE_SERVICE_DB_BACKEND)); + MDC.of(LogKeys.SHUFFLE_DB_BACKEND_NAME, dbBackend), + MDC.of(LogKeys.SHUFFLE_DB_BACKEND_KEY, Constants.SHUFFLE_SERVICE_DB_BACKEND)); executors = reloadRegisteredExecutors(db); } else { executors = Maps.newConcurrentMap(); @@ -152,8 +152,8 @@ public void registerExecutor( ExecutorShuffleInfo executorInfo) { AppExecId fullId = new AppExecId(appId, execId); logger.info("Registered executor {} with {}", - MDC.of(LogKeys.APP_EXECUTOR_ID$.MODULE$, fullId), - MDC.of(LogKeys.EXECUTOR_SHUFFLE_INFO$.MODULE$, executorInfo)); + MDC.of(LogKeys.APP_EXECUTOR_ID, fullId), + MDC.of(LogKeys.EXECUTOR_SHUFFLE_INFO, executorInfo)); try { if (db != null && AppsWithRecoveryDisabled.isRecoveryEnabledForApp(appId)) { byte[] key = dbAppExecKey(fullId); @@ -219,8 +219,8 @@ public ManagedBuffer getRddBlockData( */ public void applicationRemoved(String appId, boolean cleanupLocalDirs) { logger.info("Application {} removed, cleanupLocalDirs = {}", - MDC.of(LogKeys.APP_ID$.MODULE$, appId), - MDC.of(LogKeys.CLEANUP_LOCAL_DIRS$.MODULE$, cleanupLocalDirs)); + MDC.of(LogKeys.APP_ID, appId), + MDC.of(LogKeys.CLEANUP_LOCAL_DIRS, cleanupLocalDirs)); Iterator> it = executors.entrySet().iterator(); while (it.hasNext()) { Map.Entry entry = it.next(); @@ -235,14 +235,14 @@ public void applicationRemoved(String appId, boolean cleanupLocalDirs) { db.delete(dbAppExecKey(fullId)); } catch (IOException e) { logger.error("Error deleting {} from executor state db", e, - MDC.of(LogKeys.APP_ID$.MODULE$, appId)); + MDC.of(LogKeys.APP_ID, appId)); } } if (cleanupLocalDirs) { logger.info("Cleaning up executor {}'s {} local dirs", - MDC.of(LogKeys.APP_EXECUTOR_ID$.MODULE$, fullId), - MDC.of(LogKeys.NUM_LOCAL_DIRS$.MODULE$, executor.localDirs.length)); + MDC.of(LogKeys.APP_EXECUTOR_ID, fullId), + MDC.of(LogKeys.NUM_LOCAL_DIRS, executor.localDirs.length)); // Execute the actual deletion in a different thread, as it may take some time. directoryCleaner.execute(() -> deleteExecutorDirs(executor.localDirs)); @@ -257,18 +257,18 @@ public void applicationRemoved(String appId, boolean cleanupLocalDirs) { */ public void executorRemoved(String executorId, String appId) { logger.info("Clean up non-shuffle and non-RDD files associated with the finished executor {}", - MDC.of(LogKeys.EXECUTOR_ID$.MODULE$, executorId)); + MDC.of(LogKeys.EXECUTOR_ID, executorId)); AppExecId fullId = new AppExecId(appId, executorId); final ExecutorShuffleInfo executor = executors.get(fullId); if (executor == null) { // Executor not registered, skip clean up of the local directories. logger.info("Executor is not registered (appId={}, execId={})", - MDC.of(LogKeys.APP_ID$.MODULE$, appId), - MDC.of(LogKeys.EXECUTOR_ID$.MODULE$, executorId)); + MDC.of(LogKeys.APP_ID, appId), + MDC.of(LogKeys.EXECUTOR_ID, executorId)); } else { logger.info("Cleaning up non-shuffle and non-RDD files in executor {}'s {} local dirs", - MDC.of(LogKeys.APP_EXECUTOR_ID$.MODULE$, fullId), - MDC.of(LogKeys.NUM_LOCAL_DIRS$.MODULE$, executor.localDirs.length)); + MDC.of(LogKeys.APP_EXECUTOR_ID, fullId), + MDC.of(LogKeys.NUM_LOCAL_DIRS, executor.localDirs.length)); // Execute the actual deletion in a different thread, as it may take some time. directoryCleaner.execute(() -> deleteNonShuffleServiceServedFiles(executor.localDirs)); @@ -286,7 +286,7 @@ private void deleteExecutorDirs(String[] dirs) { logger.debug("Successfully cleaned up directory: {}", localDir); } catch (Exception e) { logger.error("Failed to delete directory: {}", e, - MDC.of(LogKeys.PATH$.MODULE$, localDir)); + MDC.of(LogKeys.PATH, localDir)); } } } @@ -309,7 +309,7 @@ private void deleteNonShuffleServiceServedFiles(String[] dirs) { localDir); } catch (Exception e) { logger.error("Failed to delete files not served by shuffle service in directory: {}", e, - MDC.of(LogKeys.PATH$.MODULE$, localDir)); + MDC.of(LogKeys.PATH, localDir)); } } } @@ -382,7 +382,7 @@ public int removeBlocks(String appId, String execId, String[] blockIds) { numRemovedBlocks++; } else { logger.warn("Failed to delete block: {}", - MDC.of(LogKeys.PATH$.MODULE$, file.getAbsolutePath())); + MDC.of(LogKeys.PATH, file.getAbsolutePath())); } } return numRemovedBlocks; @@ -484,7 +484,7 @@ static ConcurrentMap reloadRegisteredExecutors(D } AppExecId id = parseDbAppExecKey(key); logger.info("Reloading registered executors: {}", - MDC.of(LogKeys.APP_EXECUTOR_ID$.MODULE$, id)); + MDC.of(LogKeys.APP_EXECUTOR_ID, id)); ExecutorShuffleInfo shuffleInfo = mapper.readValue(e.getValue(), ExecutorShuffleInfo.class); registeredExecutors.put(id, shuffleInfo); diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index 6e9bd548f5327..7968e03f580c2 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -187,8 +187,8 @@ public ShuffleIndexInformation load(String filePath) throws IOException { db = DBProvider.initDB(dbBackend, this.recoveryFile, CURRENT_VERSION, mapper); if (db != null) { logger.info("Use {} as the implementation of {}", - MDC.of(LogKeys.SHUFFLE_DB_BACKEND_NAME$.MODULE$, dbBackend), - MDC.of(LogKeys.SHUFFLE_DB_BACKEND_KEY$.MODULE$, Constants.SHUFFLE_SERVICE_DB_BACKEND)); + MDC.of(LogKeys.SHUFFLE_DB_BACKEND_NAME, dbBackend), + MDC.of(LogKeys.SHUFFLE_DB_BACKEND_KEY, Constants.SHUFFLE_SERVICE_DB_BACKEND)); reloadAndCleanUpAppShuffleInfo(db); } this.pushMergeMetrics = new PushMergeMetrics(); @@ -234,10 +234,10 @@ AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo( if (mergePartitionsInfo == null) { logger.info("{} attempt {} shuffle {} shuffleMerge {}: creating a new shuffle " + "merge metadata", - MDC.of(LogKeys.APP_ID$.MODULE$, appShuffleInfo.appId), - MDC.of(LogKeys.APP_ATTEMPT_ID$.MODULE$, appShuffleInfo.attemptId), - MDC.of(LogKeys.SHUFFLE_ID$.MODULE$, shuffleId), - MDC.of(LogKeys.SHUFFLE_MERGE_ID$.MODULE$, shuffleMergeId)); + MDC.of(LogKeys.APP_ID, appShuffleInfo.appId), + MDC.of(LogKeys.APP_ATTEMPT_ID, appShuffleInfo.attemptId), + MDC.of(LogKeys.SHUFFLE_ID, shuffleId), + MDC.of(LogKeys.SHUFFLE_MERGE_ID, shuffleMergeId)); return new AppShuffleMergePartitionsInfo(shuffleMergeId, false); } else { int latestShuffleMergeId = mergePartitionsInfo.shuffleMergeId; @@ -256,10 +256,10 @@ AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo( shuffleId, latestShuffleMergeId); logger.info("{}: creating a new shuffle merge metadata since received " + "shuffleMergeId {} is higher than latest shuffleMergeId {}", - MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID$.MODULE$, + MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID, currentAppAttemptShuffleMergeId), - MDC.of(LogKeys.SHUFFLE_MERGE_ID$.MODULE$, shuffleMergeId), - MDC.of(LogKeys.LATEST_SHUFFLE_MERGE_ID$.MODULE$, latestShuffleMergeId)); + MDC.of(LogKeys.SHUFFLE_MERGE_ID, shuffleMergeId), + MDC.of(LogKeys.LATEST_SHUFFLE_MERGE_ID, latestShuffleMergeId)); submitCleanupTask(() -> closeAndDeleteOutdatedPartitions(currentAppAttemptShuffleMergeId, mergePartitionsInfo.shuffleMergePartitions)); @@ -293,13 +293,13 @@ AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo( } catch (IOException e) { logger.error("{} attempt {} shuffle {} shuffleMerge {}: cannot create merged shuffle " + "partition with data file {}, index file {}, and meta file {}", - MDC.of(LogKeys.APP_ID$.MODULE$, appShuffleInfo.appId), - MDC.of(LogKeys.APP_ATTEMPT_ID$.MODULE$, appShuffleInfo.attemptId), - MDC.of(LogKeys.SHUFFLE_ID$.MODULE$, shuffleId), - MDC.of(LogKeys.SHUFFLE_MERGE_ID$.MODULE$, shuffleMergeId), - MDC.of(LogKeys.DATA_FILE$.MODULE$, dataFile.getAbsolutePath()), - MDC.of(LogKeys.INDEX_FILE$.MODULE$, indexFile.getAbsolutePath()), - MDC.of(LogKeys.META_FILE$.MODULE$, metaFile.getAbsolutePath())); + MDC.of(LogKeys.APP_ID, appShuffleInfo.appId), + MDC.of(LogKeys.APP_ATTEMPT_ID, appShuffleInfo.attemptId), + MDC.of(LogKeys.SHUFFLE_ID, shuffleId), + MDC.of(LogKeys.SHUFFLE_MERGE_ID, shuffleMergeId), + MDC.of(LogKeys.DATA_FILE, dataFile.getAbsolutePath()), + MDC.of(LogKeys.INDEX_FILE, indexFile.getAbsolutePath()), + MDC.of(LogKeys.META_FILE, metaFile.getAbsolutePath())); throw new RuntimeException( String.format("Cannot initialize merged shuffle partition for appId %s shuffleId %s " + "shuffleMergeId %s reduceId %s", appShuffleInfo.appId, shuffleId, shuffleMergeId, @@ -411,8 +411,8 @@ private void removeOldApplicationAttemptsFromDb(AppShuffleInfo info) { @Override public void applicationRemoved(String appId, boolean cleanupLocalDirs) { logger.info("Application {} removed, cleanupLocalDirs = {}", - MDC.of(LogKeys.APP_ID$.MODULE$, appId), - MDC.of(LogKeys.CLEANUP_LOCAL_DIRS$.MODULE$, cleanupLocalDirs)); + MDC.of(LogKeys.APP_ID, appId), + MDC.of(LogKeys.CLEANUP_LOCAL_DIRS, cleanupLocalDirs)); // Cleanup the DB within critical section to gain the consistency between // DB and in-memory hashmap. AtomicReference ref = new AtomicReference<>(null); @@ -523,7 +523,7 @@ void removeAppAttemptPathInfoFromDB(String appId, int attemptId) { db.delete(key); } catch (Exception e) { logger.error("Failed to remove the application attempt {} local path in DB", e, - MDC.of(LogKeys.APP_ATTEMPT_ID$.MODULE$, appAttemptId)); + MDC.of(LogKeys.APP_ATTEMPT_ID, appAttemptId)); } } } @@ -593,10 +593,10 @@ void deleteMergedFiles( } } logger.info("Delete {} data files, {} index files, {} meta files for {}", - MDC.of(LogKeys.NUM_DATA_FILES$.MODULE$, dataFilesDeleteCnt), - MDC.of(LogKeys.NUM_INDEX_FILES$.MODULE$, indexFilesDeleteCnt), - MDC.of(LogKeys.NUM_META_FILES$.MODULE$, metaFilesDeleteCnt), - MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID$.MODULE$, appAttemptShuffleMergeId)); + MDC.of(LogKeys.NUM_DATA_FILES, dataFilesDeleteCnt), + MDC.of(LogKeys.NUM_INDEX_FILES, indexFilesDeleteCnt), + MDC.of(LogKeys.NUM_META_FILES, metaFilesDeleteCnt), + MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID, appAttemptShuffleMergeId)); } /** @@ -609,7 +609,7 @@ void removeAppShufflePartitionInfoFromDB(AppAttemptShuffleMergeId appAttemptShuf db.delete(getDbAppAttemptShufflePartitionKey(appAttemptShuffleMergeId)); } catch (Exception e) { logger.error("Error deleting {} from application shuffle merged partition info in DB", e, - MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID$.MODULE$, appAttemptShuffleMergeId)); + MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID, appAttemptShuffleMergeId)); } } } @@ -629,7 +629,7 @@ void deleteExecutorDirs(AppShuffleInfo appShuffleInfo) { } } catch (Exception e) { logger.error("Failed to delete directory: {}", e, - MDC.of(LogKeys.PATH$.MODULE$, localDir)); + MDC.of(LogKeys.PATH, localDir)); } } } @@ -759,10 +759,10 @@ public ByteBuffer getCompletionResponse() { @Override public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) { logger.info("{} attempt {} shuffle {} shuffleMerge {}: finalize shuffle merge", - MDC.of(LogKeys.APP_ID$.MODULE$, msg.appId), - MDC.of(LogKeys.APP_ATTEMPT_ID$.MODULE$, msg.appAttemptId), - MDC.of(LogKeys.SHUFFLE_ID$.MODULE$, msg.shuffleId), - MDC.of(LogKeys.SHUFFLE_MERGE_ID$.MODULE$, msg.shuffleMergeId)); + MDC.of(LogKeys.APP_ID, msg.appId), + MDC.of(LogKeys.APP_ATTEMPT_ID, msg.appAttemptId), + MDC.of(LogKeys.SHUFFLE_ID, msg.shuffleId), + MDC.of(LogKeys.SHUFFLE_MERGE_ID, msg.shuffleMergeId)); AppShuffleInfo appShuffleInfo = validateAndGetAppShuffleInfo(msg.appId); if (appShuffleInfo.attemptId != msg.appAttemptId) { // If finalizeShuffleMerge from a former application attempt, it is considered late, @@ -846,12 +846,12 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) { } catch (IOException ioe) { logger.warn("{} attempt {} shuffle {} shuffleMerge {}: exception while " + "finalizing shuffle partition {}. Exception message: {}", - MDC.of(LogKeys.APP_ID$.MODULE$, msg.appId), - MDC.of(LogKeys.APP_ATTEMPT_ID$.MODULE$, msg.appAttemptId), - MDC.of(LogKeys.SHUFFLE_ID$.MODULE$, msg.shuffleId), - MDC.of(LogKeys.SHUFFLE_MERGE_ID$.MODULE$, msg.shuffleMergeId), - MDC.of(LogKeys.REDUCE_ID$.MODULE$, partition.reduceId), - MDC.of(LogKeys.EXCEPTION$.MODULE$, ioe.getMessage())); + MDC.of(LogKeys.APP_ID, msg.appId), + MDC.of(LogKeys.APP_ATTEMPT_ID, msg.appAttemptId), + MDC.of(LogKeys.SHUFFLE_ID, msg.shuffleId), + MDC.of(LogKeys.SHUFFLE_MERGE_ID, msg.shuffleMergeId), + MDC.of(LogKeys.REDUCE_ID, partition.reduceId), + MDC.of(LogKeys.EXCEPTION, ioe.getMessage())); } finally { partition.cleanable.clean(); } @@ -863,10 +863,10 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) { appShuffleInfo.shuffles.get(msg.shuffleId).setReduceIds(Ints.toArray(reduceIds)); } logger.info("{} attempt {} shuffle {} shuffleMerge {}: finalization of shuffle merge completed", - MDC.of(LogKeys.APP_ID$.MODULE$, msg.appId), - MDC.of(LogKeys.APP_ATTEMPT_ID$.MODULE$, msg.appAttemptId), - MDC.of(LogKeys.SHUFFLE_ID$.MODULE$, msg.shuffleId), - MDC.of(LogKeys.SHUFFLE_MERGE_ID$.MODULE$, msg.shuffleMergeId)); + MDC.of(LogKeys.APP_ID, msg.appId), + MDC.of(LogKeys.APP_ATTEMPT_ID, msg.appAttemptId), + MDC.of(LogKeys.SHUFFLE_ID, msg.shuffleId), + MDC.of(LogKeys.SHUFFLE_MERGE_ID, msg.shuffleMergeId)); return mergeStatuses; } @@ -934,8 +934,8 @@ public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) { if (originalAppShuffleInfo.get() != null) { AppShuffleInfo appShuffleInfo = originalAppShuffleInfo.get(); logger.warn("Cleanup shuffle info and merged shuffle files for {}_{} as new " + - "application attempt registered", MDC.of(LogKeys.APP_ID$.MODULE$, appId), - MDC.of(LogKeys.APP_ATTEMPT_ID$.MODULE$, appShuffleInfo.attemptId)); + "application attempt registered", MDC.of(LogKeys.APP_ID, appId), + MDC.of(LogKeys.APP_ATTEMPT_ID, appShuffleInfo.attemptId)); // Clean up all the merge shuffle related information in the DB for the former attempt submitCleanupTask( () -> closeAndDeletePartitionsIfNeeded(appShuffleInfo, true) @@ -992,12 +992,12 @@ private void shutdownMergedShuffleCleanerNow() { List unfinishedTasks = mergedShuffleCleaner.shutdownNow(); logger.warn("There are still {} tasks not completed in mergedShuffleCleaner " + "after {} ms.", - MDC.of(LogKeys.COUNT$.MODULE$, unfinishedTasks.size()), - MDC.of(LogKeys.TIMEOUT$.MODULE$, cleanerShutdownTimeout * 1000L)); + MDC.of(LogKeys.COUNT, unfinishedTasks.size()), + MDC.of(LogKeys.TIMEOUT, cleanerShutdownTimeout * 1000L)); // Wait a while for tasks to respond to being cancelled if (!mergedShuffleCleaner.awaitTermination(cleanerShutdownTimeout, TimeUnit.SECONDS)) { logger.warn("mergedShuffleCleaner did not terminate in {} ms.", - MDC.of(LogKeys.TIMEOUT$.MODULE$, cleanerShutdownTimeout * 1000L)); + MDC.of(LogKeys.TIMEOUT, cleanerShutdownTimeout * 1000L)); } } catch (InterruptedException ignored) { Thread.currentThread().interrupt(); @@ -1017,7 +1017,7 @@ private void writeAppPathsInfoToDb(String appId, int attemptId, AppPathsInfo app db.put(key, value); } catch (Exception e) { logger.error("Error saving registered app paths info for {}", e, - MDC.of(LogKeys.APP_ATTEMPT_ID$.MODULE$, appAttemptId)); + MDC.of(LogKeys.APP_ATTEMPT_ID, appAttemptId)); } } } @@ -1035,7 +1035,7 @@ private void writeAppAttemptShuffleMergeInfoToDB( db.put(dbKey, new byte[0]); } catch (Exception e) { logger.error("Error saving active app shuffle partition {}", e, - MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID$.MODULE$, appAttemptShuffleMergeId)); + MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID, appAttemptShuffleMergeId)); } } } @@ -1137,7 +1137,7 @@ List reloadActiveAppAttemptsPathInfo(DB db) throws IOException { dbKeysToBeRemoved.add(getDbAppAttemptPathsKey(existingAppAttemptId)); } catch (IOException e) { logger.error("Failed to get the DB key for {}", e, - MDC.of(LogKeys.APP_ATTEMPT_ID$.MODULE$, existingAppAttemptId)); + MDC.of(LogKeys.APP_ATTEMPT_ID, existingAppAttemptId)); } } return new AppShuffleInfo( @@ -1187,7 +1187,7 @@ List reloadFinalizedAppAttemptsShuffleMergeInfo(DB db) throws IOExceptio getDbAppAttemptShufflePartitionKey(appAttemptShuffleMergeId)); } catch (Exception e) { logger.error("Error getting the DB key for {}", e, MDC.of( - LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID$.MODULE$, appAttemptShuffleMergeId)); + LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID, appAttemptShuffleMergeId)); } } return new AppShuffleMergePartitionsInfo(partitionId.shuffleMergeId, true); @@ -1216,7 +1216,7 @@ void removeOutdatedKeyValuesInDB(List dbKeysToBeRemoved) { db.delete(key); } catch (Exception e) { logger.error("Error deleting dangling key {} in DB", e, - MDC.of(LogKeys.KEY$.MODULE$, key)); + MDC.of(LogKeys.KEY, key)); } } ); @@ -1599,7 +1599,7 @@ public void onComplete(String streamId) throws IOException { public void onFailure(String streamId, Throwable throwable) throws IOException { if (ERROR_HANDLER.shouldLogError(throwable)) { logger.error("Encountered issue when merging {}", throwable, - MDC.of(LogKeys.STREAM_ID$.MODULE$, streamId)); + MDC.of(LogKeys.STREAM_ID, streamId)); } else { logger.debug("Encountered issue when merging {}", streamId, throwable); } @@ -1860,8 +1860,8 @@ void updateChunkInfo(long chunkOffset, int mapIndex) throws IOException { indexMetaUpdateFailed = false; } catch (IOException ioe) { logger.warn("{} reduceId {} update to index/meta failed", - MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID$.MODULE$, appAttemptShuffleMergeId), - MDC.of(LogKeys.REDUCE_ID$.MODULE$, reduceId)); + MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID, appAttemptShuffleMergeId), + MDC.of(LogKeys.REDUCE_ID, reduceId)); indexMetaUpdateFailed = true; // Any exception here is propagated to the caller and the caller can decide whether to // abort or not. @@ -1913,8 +1913,8 @@ private void finalizePartition() throws IOException { private void deleteAllFiles() { if (!dataFile.delete()) { logger.info("Error deleting data file for {} reduceId {}", - MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID$.MODULE$, appAttemptShuffleMergeId), - MDC.of(LogKeys.REDUCE_ID$.MODULE$, reduceId)); + MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID, appAttemptShuffleMergeId), + MDC.of(LogKeys.REDUCE_ID, reduceId)); } metaFile.delete(); indexFile.delete(); @@ -1983,22 +1983,22 @@ private void closeAllFiles( } } catch (IOException ioe) { logger.warn("Error closing data channel for {} reduceId {}", - MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID$.MODULE$, appAttemptShuffleMergeId), - MDC.of(LogKeys.REDUCE_ID$.MODULE$, reduceId)); + MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID, appAttemptShuffleMergeId), + MDC.of(LogKeys.REDUCE_ID, reduceId)); } try { metaFile.close(); } catch (IOException ioe) { logger.warn("Error closing meta file for {} reduceId {}", - MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID$.MODULE$, appAttemptShuffleMergeId), - MDC.of(LogKeys.REDUCE_ID$.MODULE$, reduceId)); + MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID, appAttemptShuffleMergeId), + MDC.of(LogKeys.REDUCE_ID, reduceId)); } try { indexFile.close(); } catch (IOException ioe) { logger.warn("Error closing index file for {} reduceId {}", - MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID$.MODULE$, appAttemptShuffleMergeId), - MDC.of(LogKeys.REDUCE_ID$.MODULE$, reduceId)); + MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID, appAttemptShuffleMergeId), + MDC.of(LogKeys.REDUCE_ID, reduceId)); } } } @@ -2043,9 +2043,9 @@ private AppPathsInfo( this.subDirsPerLocalDir = subDirsPerLocalDir; if (logger.isInfoEnabled()) { logger.info("Updated active local dirs {} and sub dirs {} for application {}", - MDC.of(LogKeys.PATHS$.MODULE$, Arrays.toString(activeLocalDirs)), - MDC.of(LogKeys.NUM_SUB_DIRS$.MODULE$, subDirsPerLocalDir), - MDC.of(LogKeys.APP_ID$.MODULE$, appId)); + MDC.of(LogKeys.PATHS, Arrays.toString(activeLocalDirs)), + MDC.of(LogKeys.NUM_SUB_DIRS, subDirsPerLocalDir), + MDC.of(LogKeys.APP_ID, appId)); } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockTransferor.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockTransferor.java index 31c454f63a92e..3d7b5bb18f9e5 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockTransferor.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockTransferor.java @@ -182,13 +182,13 @@ private void transferAllOutstanding() { } catch (Exception e) { if (numRetries > 0) { logger.error("Exception while beginning {} of {} outstanding blocks (after {} retries)", e, - MDC.of(LogKeys.TRANSFER_TYPE$.MODULE$, listener.getTransferType()), - MDC.of(LogKeys.NUM_BLOCKS$.MODULE$, blockIdsToTransfer.length), - MDC.of(LogKeys.NUM_RETRY$.MODULE$, numRetries)); + MDC.of(LogKeys.TRANSFER_TYPE, listener.getTransferType()), + MDC.of(LogKeys.NUM_BLOCKS, blockIdsToTransfer.length), + MDC.of(LogKeys.NUM_RETRY, numRetries)); } else { logger.error("Exception while beginning {} of {} outstanding blocks", e, - MDC.of(LogKeys.TRANSFER_TYPE$.MODULE$, listener.getTransferType()), - MDC.of(LogKeys.NUM_BLOCKS$.MODULE$, blockIdsToTransfer.length)); + MDC.of(LogKeys.TRANSFER_TYPE, listener.getTransferType()), + MDC.of(LogKeys.NUM_BLOCKS, blockIdsToTransfer.length)); } if (shouldRetry(e) && initiateRetry(e)) { // successfully initiated a retry @@ -216,11 +216,11 @@ synchronized boolean initiateRetry(Throwable e) { currentListener = new RetryingBlockTransferListener(); logger.info("Retrying {} ({}/{}) for {} outstanding blocks after {} ms", - MDC.of(LogKeys.TRANSFER_TYPE$.MODULE$, listener.getTransferType()), - MDC.of(LogKeys.NUM_RETRY$.MODULE$, retryCount), - MDC.of(LogKeys.MAX_ATTEMPTS$.MODULE$, maxRetries), - MDC.of(LogKeys.NUM_BLOCKS$.MODULE$, outstandingBlocksIds.size()), - MDC.of(LogKeys.RETRY_WAIT_TIME$.MODULE$, retryWaitTime)); + MDC.of(LogKeys.TRANSFER_TYPE, listener.getTransferType()), + MDC.of(LogKeys.NUM_RETRY, retryCount), + MDC.of(LogKeys.MAX_ATTEMPTS, maxRetries), + MDC.of(LogKeys.NUM_BLOCKS, outstandingBlocksIds.size()), + MDC.of(LogKeys.RETRY_WAIT_TIME, retryWaitTime)); try { executorService.execute(() -> { @@ -311,9 +311,9 @@ private void handleBlockTransferFailure(String blockId, Throwable exception) { } else { if (errorHandler.shouldLogError(exception)) { logger.error("Failed to {} block {}, and will not retry ({} retries)", exception, - MDC.of(LogKeys.TRANSFER_TYPE$.MODULE$, listener.getTransferType()), - MDC.of(LogKeys.BLOCK_ID$.MODULE$, blockId), - MDC.of(LogKeys.NUM_RETRY$.MODULE$,retryCount)); + MDC.of(LogKeys.TRANSFER_TYPE, listener.getTransferType()), + MDC.of(LogKeys.BLOCK_ID, blockId), + MDC.of(LogKeys.NUM_RETRY,retryCount)); } else { logger.debug( String.format("Failed to %s block %s, and will not retry (%s retries)", diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/checksum/ShuffleChecksumHelper.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/checksum/ShuffleChecksumHelper.java index 62fcda701d948..659bebd3dd57d 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/checksum/ShuffleChecksumHelper.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/checksum/ShuffleChecksumHelper.java @@ -156,7 +156,7 @@ public static Cause diagnoseCorruption( } catch (FileNotFoundException e) { // Even if checksum is enabled, a checksum file may not exist if error throws during writing. logger.warn("Checksum file {} doesn't exit", - MDC.of(LogKeys.PATH$.MODULE$, checksumFile.getName())); + MDC.of(LogKeys.PATH, checksumFile.getName())); cause = Cause.UNKNOWN_ISSUE; } catch (Exception e) { logger.warn("Unable to diagnose shuffle block corruption", e); @@ -169,9 +169,9 @@ public static Cause diagnoseCorruption( checksumByReader, checksumByWriter, checksumByReCalculation); } else { logger.info("Shuffle corruption diagnosis took {} ms, checksum file {}, cause {}", - MDC.of(LogKeys.TIME$.MODULE$, duration), - MDC.of(LogKeys.PATH$.MODULE$, checksumFile.getAbsolutePath()), - MDC.of(LogKeys.REASON$.MODULE$, cause)); + MDC.of(LogKeys.TIME, duration), + MDC.of(LogKeys.PATH, checksumFile.getAbsolutePath()), + MDC.of(LogKeys.REASON, cause)); } return cause; } diff --git a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index c0f795b98e062..cc8a0cd719fd9 100644 --- a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -238,7 +238,7 @@ protected void serviceInit(Configuration externalConf) throws Exception { .getResource(SHUFFLE_SERVICE_CONF_OVERLAY_RESOURCE_NAME); if (confOverlayUrl != null) { logger.info("Initializing Spark YARN shuffle service with configuration overlay from {}", - MDC.of(LogKeys.SHUFFLE_SERVICE_CONF_OVERLAY_URL$.MODULE$, confOverlayUrl)); + MDC.of(LogKeys.SHUFFLE_SERVICE_CONF_OVERLAY_URL, confOverlayUrl)); _conf.addResource(confOverlayUrl); } @@ -263,8 +263,8 @@ protected void serviceInit(Configuration externalConf) throws Exception { DBBackend.ROCKSDB.name()); dbBackend = DBBackend.byName(dbBackendName); logger.info("Use {} as the implementation of {}", - MDC.of(LogKeys.SHUFFLE_DB_BACKEND_NAME$.MODULE$, dbBackend), - MDC.of(LogKeys.SHUFFLE_DB_BACKEND_KEY$.MODULE$, Constants.SHUFFLE_SERVICE_DB_BACKEND)); + MDC.of(LogKeys.SHUFFLE_DB_BACKEND_NAME, dbBackend), + MDC.of(LogKeys.SHUFFLE_DB_BACKEND_KEY, Constants.SHUFFLE_SERVICE_DB_BACKEND)); } try { @@ -328,12 +328,12 @@ protected void serviceInit(Configuration externalConf) throws Exception { "PushBasedShuffleMergeManager", "Metrics on the push-based shuffle merge manager", mergeManagerMetrics); logger.info("Registered metrics with Hadoop's DefaultMetricsSystem using namespace '{}'", - MDC.of(LogKeys.SHUFFLE_SERVICE_METRICS_NAMESPACE$.MODULE$, metricsNamespace)); + MDC.of(LogKeys.SHUFFLE_SERVICE_METRICS_NAMESPACE, metricsNamespace)); logger.info("Started YARN shuffle service for Spark on port {}. Authentication is {}. " + - "Registered executor file is {}", MDC.of(LogKeys.PORT$.MODULE$, port), - MDC.of(LogKeys.AUTH_ENABLED$.MODULE$, authEnabledString), - MDC.of(LogKeys.REGISTERED_EXECUTOR_FILE$.MODULE$, registeredExecutorFile)); + "Registered executor file is {}", MDC.of(LogKeys.PORT, port), + MDC.of(LogKeys.AUTH_ENABLED, authEnabledString), + MDC.of(LogKeys.REGISTERED_EXECUTOR_FILE, registeredExecutorFile)); } catch (Exception e) { if (stopOnFailure) { throw e; @@ -367,7 +367,7 @@ static MergedShuffleFileManager newMergedShuffleFileManagerInstance( .newInstance(conf, mergeManagerFile); } catch (Exception e) { defaultSparkLogger.error("Unable to create an instance of {}", - MDC.of(LogKeys.CLASS_NAME$.MODULE$, mergeManagerImplClassName)); + MDC.of(LogKeys.CLASS_NAME, mergeManagerImplClassName)); return new NoOpMergedShuffleFileManager(conf, mergeManagerFile); } } @@ -431,7 +431,7 @@ public void initializeApplication(ApplicationInitializationContext context) { if (metadataStorageVal != null && (Boolean) metadataStorageVal) { AppsWithRecoveryDisabled.disableRecoveryOfApp(appId); logger.info("Disabling metadata persistence for application {}", - MDC.of(LogKeys.APP_ID$.MODULE$, appId)); + MDC.of(LogKeys.APP_ID, appId)); } } catch (IOException ioe) { logger.warn("Unable to parse application data for service: " + payload); @@ -455,7 +455,7 @@ public void initializeApplication(ApplicationInitializationContext context) { } } catch (Exception e) { logger.error("Exception when initializing application {}", e, - MDC.of(LogKeys.APP_ID$.MODULE$, appId)); + MDC.of(LogKeys.APP_ID, appId)); } } @@ -470,7 +470,7 @@ public void stopApplication(ApplicationTerminationContext context) { db.delete(dbAppKey(fullId)); } catch (IOException e) { logger.error("Error deleting {} from executor state db", e, - MDC.of(LogKeys.APP_ID$.MODULE$, appId)); + MDC.of(LogKeys.APP_ID, appId)); } } secretManager.unregisterApp(appId); @@ -478,7 +478,7 @@ public void stopApplication(ApplicationTerminationContext context) { blockHandler.applicationRemoved(appId, false /* clean up local dirs */); } catch (Exception e) { logger.error("Exception when stopping application {}", e, - MDC.of(LogKeys.APP_ID$.MODULE$, appId)); + MDC.of(LogKeys.APP_ID, appId)); } finally { AppsWithRecoveryDisabled.removeApp(appId); } @@ -487,13 +487,13 @@ public void stopApplication(ApplicationTerminationContext context) { @Override public void initializeContainer(ContainerInitializationContext context) { ContainerId containerId = context.getContainerId(); - logger.info("Initializing container {}", MDC.of(LogKeys.CONTAINER_ID$.MODULE$, containerId)); + logger.info("Initializing container {}", MDC.of(LogKeys.CONTAINER_ID, containerId)); } @Override public void stopContainer(ContainerTerminationContext context) { ContainerId containerId = context.getContainerId(); - logger.info("Stopping container {}", MDC.of(LogKeys.CONTAINER_ID$.MODULE$, containerId)); + logger.info("Stopping container {}", MDC.of(LogKeys.CONTAINER_ID, containerId)); } /** @@ -575,8 +575,8 @@ protected File initRecoveryDb(String dbName) { } catch (Exception e) { // Fail to move recovery file to new path, just continue on with new DB location logger.error("Failed to move recovery file {} to the path {}", e, - MDC.of(LogKeys.SHUFFLE_MERGE_RECOVERY_FILE$.MODULE$, dbName), - MDC.of(LogKeys.PATH$.MODULE$, _recoveryPath.toString())); + MDC.of(LogKeys.SHUFFLE_MERGE_RECOVERY_FILE, dbName), + MDC.of(LogKeys.PATH, _recoveryPath.toString())); } } return new File(newLoc.toUri().getPath()); diff --git a/common/utils/src/main/java/org/apache/spark/internal/LogKey.java b/common/utils/src/main/java/org/apache/spark/internal/LogKey.java new file mode 100644 index 0000000000000..0bd0fecb43976 --- /dev/null +++ b/common/utils/src/main/java/org/apache/spark/internal/LogKey.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.internal; + +/** + * All structured logging `keys` used in `MDC` must be extends `LogKey` + *

+ * + * `LogKey`s serve as identifiers for mapped diagnostic contexts (MDC) within logs. + * Follow these guidelines when adding a new LogKey: + *

    + *
  • + * Define all structured logging keys in `LogKeys.java`, and sort them alphabetically for + * ease of search. + *
  • + *
  • + * Use `UPPER_SNAKE_CASE` for key names. + *
  • + *
  • + * Key names should be both simple and broad, yet include specific identifiers like `STAGE_ID`, + * `TASK_ID`, and `JOB_ID` when needed for clarity. For instance, use `MAX_ATTEMPTS` as a + * general key instead of creating separate keys for each scenario such as + * `EXECUTOR_STATE_SYNC_MAX_ATTEMPTS` and `MAX_TASK_FAILURES`. + * This balances simplicity with the detail needed for effective logging. + *
  • + *
  • + * Use abbreviations in names if they are widely understood, + * such as `APP_ID` for APPLICATION_ID, and `K8S` for KUBERNETES. + *
  • + *
  • + * For time-related keys, use milliseconds as the unit of time. + *
  • + *
+ */ +public interface LogKey { + String name(); +} diff --git a/common/utils/src/main/java/org/apache/spark/internal/LogKeys.java b/common/utils/src/main/java/org/apache/spark/internal/LogKeys.java new file mode 100644 index 0000000000000..0fd3627bbac1d --- /dev/null +++ b/common/utils/src/main/java/org/apache/spark/internal/LogKeys.java @@ -0,0 +1,889 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.internal; + +/** + * Various keys used for mapped diagnostic contexts(MDC) in logging. All structured logging keys + * should be defined here for standardization. + */ +public enum LogKeys implements LogKey { + ACCUMULATOR_ID, + ACL_ENABLED, + ACTUAL_NUM_FILES, + ACTUAL_PARTITION_COLUMN, + ADDED_JARS, + ADMIN_ACLS, + ADMIN_ACL_GROUPS, + ADVISORY_TARGET_SIZE, + AGGREGATE_FUNCTIONS, + ALIGNED_FROM_TIME, + ALIGNED_TO_TIME, + ALPHA, + ANALYSIS_ERROR, + ANTLR_DFA_CACHE_DELTA, + ANTLR_DFA_CACHE_SIZE, + APP_ATTEMPT_ID, + APP_ATTEMPT_SHUFFLE_MERGE_ID, + APP_DESC, + APP_EXECUTOR_ID, + APP_ID, + APP_NAME, + APP_STATE, + ARCHIVE_NAME, + ARGS, + ARTIFACTS, + ARTIFACT_ID, + ATTRIBUTE_MAP, + AUTH_ENABLED, + AVG_BATCH_PROC_TIME, + BACKUP_FILE, + BARRIER_EPOCH, + BARRIER_ID, + BATCH_ID, + BATCH_NAME, + BATCH_TIMES, + BATCH_TIMESTAMP, + BATCH_WRITE, + BIND_ADDRESS, + BLOCK_GENERATOR_STATUS, + BLOCK_ID, + BLOCK_IDS, + BLOCK_MANAGER_ID, + BLOCK_MANAGER_IDS, + BLOCK_TYPE, + BOOT, + BOOTSTRAP_TIME, + BOOT_TIME, + BROADCAST, + BROADCAST_ID, + BROADCAST_OUTPUT_STATUS_SIZE, + BUCKET, + BYTECODE_SIZE, + BYTE_BUFFER, + BYTE_SIZE, + CACHED_TABLE_PARTITION_METADATA_SIZE, + CACHE_AUTO_REMOVED_SIZE, + CACHE_SIZE, + CACHE_UNTIL_HIGHEST_CONSUMED_SIZE, + CACHE_UNTIL_LAST_PRODUCED_SIZE, + CALL_SITE_LONG_FORM, + CALL_SITE_SHORT_FORM, + CANCEL_FUTURE_JOBS, + CATALOG_NAME, + CATEGORICAL_FEATURES, + CHECKPOINT_FILE, + CHECKPOINT_INTERVAL, + CHECKPOINT_LOCATION, + CHECKPOINT_PATH, + CHECKPOINT_ROOT, + CHECKPOINT_TIME, + CHOSEN_WATERMARK, + CLASSIFIER, + CLASS_LOADER, + CLASS_NAME, + CLASS_PATH, + CLASS_PATHS, + CLAUSES, + CLEANUP_LOCAL_DIRS, + CLUSTER_CENTROIDS, + CLUSTER_ID, + CLUSTER_LABEL, + CLUSTER_LEVEL, + CLUSTER_WEIGHT, + CODE, + CODEC_LEVEL, + CODEC_NAME, + CODEGEN_STAGE_ID, + COLUMN_DATA_TYPE_SOURCE, + COLUMN_DATA_TYPE_TARGET, + COLUMN_DEFAULT_VALUE, + COLUMN_NAME, + COLUMN_NAMES, + COMMAND, + COMMAND_OUTPUT, + COMMITTED_VERSION, + COMPACT_INTERVAL, + COMPONENT, + COMPUTE, + CONFIG, + CONFIG2, + CONFIG3, + CONFIG4, + CONFIG5, + CONFIG_DEPRECATION_MESSAGE, + CONFIG_KEY_UPDATED, + CONFIG_VERSION, + CONSUMER, + CONTAINER, + CONTAINER_ID, + CONTAINER_STATE, + CONTEXT, + COST, + COUNT, + CREATED_POOL_NAME, + CREATION_SITE, + CREDENTIALS_RENEWAL_INTERVAL_RATIO, + CROSS_VALIDATION_METRIC, + CROSS_VALIDATION_METRICS, + CSV_HEADER_COLUMN_NAME, + CSV_HEADER_COLUMN_NAMES, + CSV_HEADER_LENGTH, + CSV_SCHEMA_FIELD_NAME, + CSV_SCHEMA_FIELD_NAMES, + CSV_SOURCE, + CURRENT_BATCH_ID, + CURRENT_DISK_SIZE, + CURRENT_FILE, + CURRENT_MEMORY_SIZE, + CURRENT_PATH, + CURRENT_TIME, + DATA, + DATABASE_NAME, + DATAFRAME_CACHE_ENTRY, + DATAFRAME_ID, + DATA_FILE, + DATA_SOURCE, + DATA_SOURCES, + DEFAULT_COMPACT_INTERVAL, + DEFAULT_ISOLATION_LEVEL, + DEFAULT_NAME, + DEFAULT_VALUE, + DELAY, + DELEGATE, + DELTA, + DEPRECATED_KEY, + DERIVATIVE, + DESCRIPTION, + DESIRED_NUM_PARTITIONS, + DESIRED_TREE_DEPTH, + DESTINATION_PATH, + DFS_FILE, + DIFF_DELTA, + DIVISIBLE_CLUSTER_INDICES_SIZE, + DRIVER_ID, + DRIVER_JVM_MEMORY, + DRIVER_MEMORY_SIZE, + DRIVER_STATE, + DROPPED_PARTITIONS, + DSTREAM, + DURATION, + EARLIEST_LOADED_VERSION, + EFFECTIVE_STORAGE_LEVEL, + ELAPSED_TIME, + ENCODING, + ENDPOINT_NAME, + END_INDEX, + END_POINT, + END_VERSION, + ENGINE, + EPOCH, + ERROR, + ESTIMATOR_PARAM_MAP, + EVALUATED_FILTERS, + EVENT, + EVENT_LOG_DESTINATION, + EVENT_LOOP, + EVENT_NAME, + EVENT_QUEUE, + EXCEPTION, + EXECUTE_INFO, + EXECUTE_KEY, + EXECUTION_MEMORY_SIZE, + EXECUTION_PLAN_LEAVES, + EXECUTOR_BACKEND, + EXECUTOR_ENVS, + EXECUTOR_ENV_REGEX, + EXECUTOR_ID, + EXECUTOR_IDS, + EXECUTOR_LAUNCH_COMMANDS, + EXECUTOR_MEMORY_OFFHEAP, + EXECUTOR_MEMORY_OVERHEAD_SIZE, + EXECUTOR_MEMORY_SIZE, + EXECUTOR_RESOURCES, + EXECUTOR_SHUFFLE_INFO, + EXECUTOR_STATE, + EXECUTOR_TIMEOUT, + EXECUTOR_USER_CLASS_PATH_FIRST, + EXEC_AMOUNT, + EXISTING_FILE, + EXISTING_PATH, + EXIT_CODE, + EXPECTED_NUM_FILES, + EXPECTED_PARTITION_COLUMN, + EXPIRY_TIMESTAMP, + EXPR, + EXPR_TERMS, + EXTENDED_EXPLAIN_GENERATOR, + FAILED_STAGE, + FAILED_STAGE_NAME, + FAILURES, + FALLBACK_VERSION, + FEATURE_COLUMN, + FEATURE_DIMENSION, + FEATURE_NAME, + FETCH_SIZE, + FIELD_NAME, + FIELD_TYPE, + FILES, + FILE_ABSOLUTE_PATH, + FILE_END_OFFSET, + FILE_FORMAT, + FILE_FORMAT2, + FILE_LENGTH_XATTR, + FILE_MODIFICATION_TIME, + FILE_NAME, + FILE_NAME2, + FILE_NAME3, + FILE_NAMES, + FILE_START_OFFSET, + FILE_SYSTEM, + FILE_VERSION, + FILTER, + FINAL_CONTEXT, + FINAL_OUTPUT_PATH, + FINAL_PATH, + FINISH_TIME, + FINISH_TRIGGER_DURATION, + FLOW_NAME, + FREE_MEMORY_SIZE, + FROM_OFFSET, + FROM_TIME, + FS_DATA_OUTPUT_STREAM, + FUNCTION_NAME, + FUNCTION_PARAM, + GLOBAL_INIT_FILE, + GLOBAL_WATERMARK, + GROUP_BY_EXPRS, + GROUP_ID, + HADOOP_VERSION, + HASH_JOIN_KEYS, + HASH_MAP_SIZE, + HEARTBEAT, + HEARTBEAT_INTERVAL, + HISTORY_DIR, + HIVE_CLIENT_VERSION, + HIVE_METASTORE_VERSION, + HIVE_OPERATION_STATE, + HIVE_OPERATION_TYPE, + HOST, + HOSTS, + HOST_LOCAL_BLOCKS_SIZE, + HOST_PORT, + HOST_PORT2, + HUGE_METHOD_LIMIT, + HYBRID_STORE_DISK_BACKEND, + IDENTIFIER, + INCOMPATIBLE_TYPES, + INDEX, + INDEX_FILE, + INDEX_NAME, + INFERENCE_MODE, + INIT, + INITIAL_CAPACITY, + INITIAL_HEARTBEAT_INTERVAL, + INIT_MODE, + INIT_TIME, + INPUT, + INPUT_SPLIT, + INTEGRAL, + INTERRUPT_THREAD, + INTERVAL, + INVALID_PARAMS, + ISOLATION_LEVEL, + ISSUE_DATE, + IS_NETWORK_REQUEST_DONE, + JAR_ENTRY, + JAR_MESSAGE, + JAR_URL, + JAVA_VERSION, + JAVA_VM_NAME, + JOB_ID, + JOIN_CONDITION, + JOIN_CONDITION_SUB_EXPR, + JOIN_TYPE, + K8S_CONTEXT, + KEY, + KEY2, + KEYTAB, + KEYTAB_FILE, + KILL_EXECUTORS, + KINESIS_REASON, + LABEL_COLUMN, + LARGEST_CLUSTER_INDEX, + LAST_ACCESS_TIME, + LAST_COMMITTED_CHECKPOINT_ID, + LAST_COMMIT_BASED_CHECKPOINT_ID, + LAST_VALID_TIME, + LATEST_BATCH_ID, + LATEST_COMMITTED_BATCH_ID, + LATEST_SHUFFLE_MERGE_ID, + LEARNING_RATE, + LEFT_EXPR, + LEFT_LOGICAL_PLAN_STATS_SIZE_IN_BYTES, + LINE, + LINEAGE, + LINE_NUM, + LISTENER, + LOADED_CHECKPOINT_ID, + LOADED_VERSION, + LOAD_FACTOR, + LOAD_TIME, + LOCALE, + LOCAL_BLOCKS_SIZE, + LOCAL_SCRATCH_DIR, + LOCATION, + LOGICAL_PLAN, + LOGICAL_PLAN_COLUMNS, + LOGICAL_PLAN_LEAVES, + LOG_ID, + LOG_LEVEL, + LOG_OFFSET, + LOG_TYPE, + LOSSES, + LOWER_BOUND, + MALFORMATTED_STRING, + MAP_ID, + MASTER_URL, + MAX_ATTEMPTS, + MAX_CACHE_UNTIL_HIGHEST_CONSUMED_SIZE, + MAX_CACHE_UNTIL_LAST_PRODUCED_SIZE, + MAX_CAPACITY, + MAX_CATEGORIES, + MAX_EXECUTOR_FAILURES, + MAX_FILE_VERSION, + MAX_JVM_METHOD_PARAMS_LENGTH, + MAX_MEMORY_SIZE, + MAX_METHOD_CODE_SIZE, + MAX_NUM_BINS, + MAX_NUM_CHUNKS, + MAX_NUM_FILES, + MAX_NUM_LOG_POLICY, + MAX_NUM_PARTITIONS, + MAX_NUM_POSSIBLE_BINS, + MAX_NUM_ROWS_IN_MEMORY_BUFFER, + MAX_SEEN_VERSION, + MAX_SERVICE_NAME_LENGTH, + MAX_SIZE, + MAX_SLOTS, + MAX_SPLIT_BYTES, + MAX_TABLE_PARTITION_METADATA_SIZE, + MEMORY_CONSUMER, + MEMORY_POOL_NAME, + MEMORY_SIZE, + MEMORY_THRESHOLD_SIZE, + MERGE_DIR_NAME, + MESSAGE, + METADATA, + METADATA_DIRECTORY, + METADATA_JSON, + META_FILE, + METHOD_NAME, + METHOD_PARAM_TYPES, + METRICS_JSON, + METRIC_NAME, + MINI_BATCH_FRACTION, + MIN_COMPACTION_BATCH_ID, + MIN_NUM_FREQUENT_PATTERN, + MIN_POINT_PER_CLUSTER, + MIN_RATE, + MIN_SEEN_VERSION, + MIN_SHARE, + MIN_SIZE, + MIN_TIME, + MIN_VERSIONS_TO_DELETE, + MIN_VERSION_NUM, + MISSING_PARENT_STAGES, + MODEL_WEIGHTS, + MODIFY_ACLS, + MODIFY_ACLS_GROUPS, + MODULE_NAME, + NAME, + NAMESPACE, + NETWORK_IF, + NEW_FEATURE_COLUMN_NAME, + NEW_LABEL_COLUMN_NAME, + NEW_PATH, + NEW_RDD_ID, + NEW_STATE, + NEW_VALUE, + NEXT_RENEWAL_TIME, + NODES, + NODE_LOCATION, + NON_BUILT_IN_CONNECTORS, + NORM, + NUM_ADDED_PARTITIONS, + NUM_APPS, + NUM_ATTEMPT, + NUM_BATCHES, + NUM_BIN, + NUM_BLOCKS, + NUM_BLOCK_IDS, + NUM_BROADCAST_BLOCK, + NUM_BYTES, + NUM_BYTES_CURRENT, + NUM_BYTES_EVICTED, + NUM_BYTES_MAX, + NUM_BYTES_TO_FREE, + NUM_BYTES_TO_WARN, + NUM_BYTES_USED, + NUM_CATEGORIES, + NUM_CHECKSUM_FILE, + NUM_CHUNKS, + NUM_CLASSES, + NUM_COEFFICIENTS, + NUM_COLUMNS, + NUM_CONCURRENT_WRITER, + NUM_CORES, + NUM_DATA_FILE, + NUM_DATA_FILES, + NUM_DECOMMISSIONED, + NUM_DRIVERS, + NUM_DROPPED_PARTITIONS, + NUM_EFFECTIVE_RULE_OF_RUNS, + NUM_ELEMENTS_SPILL_RECORDS, + NUM_ELEMENTS_SPILL_THRESHOLD, + NUM_EVENTS, + NUM_EXAMPLES, + NUM_EXECUTORS, + NUM_EXECUTORS_EXITED, + NUM_EXECUTORS_KILLED, + NUM_EXECUTOR_CORES, + NUM_EXECUTOR_CORES_REMAINING, + NUM_EXECUTOR_CORES_TOTAL, + NUM_EXECUTOR_DESIRED, + NUM_EXECUTOR_LAUNCH, + NUM_EXECUTOR_TARGET, + NUM_FAILURES, + NUM_FEATURES, + NUM_FILES, + NUM_FILES_COPIED, + NUM_FILES_FAILED_TO_DELETE, + NUM_FILES_REUSED, + NUM_FREQUENT_ITEMS, + NUM_HOST_LOCAL_BLOCKS, + NUM_INDEX_FILE, + NUM_INDEX_FILES, + NUM_ITERATIONS, + NUM_KAFKA_PULLS, + NUM_KAFKA_RECORDS_PULLED, + NUM_LAGGING_STORES, + NUM_LEADING_SINGULAR_VALUES, + NUM_LEFT_PARTITION_VALUES, + NUM_LOADED_ENTRIES, + NUM_LOCAL_BLOCKS, + NUM_LOCAL_DIRS, + NUM_LOCAL_FREQUENT_PATTERN, + NUM_MERGERS, + NUM_MERGER_LOCATIONS, + NUM_META_FILES, + NUM_NODES, + NUM_PARTITIONS, + NUM_PARTITIONS2, + NUM_PATHS, + NUM_PEERS, + NUM_PEERS_REPLICATED_TO, + NUM_PEERS_TO_REPLICATE_TO, + NUM_PENDING_LAUNCH_TASKS, + NUM_POD, + NUM_POD_SHARED_SLOT, + NUM_POD_TARGET, + NUM_POINT, + NUM_PREFIXES, + NUM_PRUNED, + NUM_PUSH_MERGED_LOCAL_BLOCKS, + NUM_RECEIVERS, + NUM_RECORDS_READ, + NUM_RELEASED_LOCKS, + NUM_REMAINED, + NUM_REMOTE_BLOCKS, + NUM_REMOVED_WORKERS, + NUM_REPLICAS, + NUM_REQUESTS, + NUM_REQUEST_SYNC_TASK, + NUM_RESOURCE_SLOTS, + NUM_RETRIES, + NUM_RETRY, + NUM_RIGHT_PARTITION_VALUES, + NUM_ROWS, + NUM_RULE_OF_RUNS, + NUM_SEQUENCES, + NUM_SKIPPED, + NUM_SLOTS, + NUM_SPILLS, + NUM_SPILL_WRITERS, + NUM_SUB_DIRS, + NUM_SUCCESSFUL_TASKS, + NUM_TASKS, + NUM_TASK_CPUS, + NUM_TRAIN_WORD, + NUM_UNFINISHED_DECOMMISSIONED, + NUM_VERSIONS_RETAIN, + NUM_WEIGHTED_EXAMPLES, + NUM_WORKERS, + OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD, + OBJECT_ID, + OFFSET, + OFFSETS, + OFFSET_SEQUENCE_METADATA, + OLD_BLOCK_MANAGER_ID, + OLD_GENERATION_GC, + OLD_VALUE, + OPEN_COST_IN_BYTES, + OPERATION, + OPERATION_HANDLE, + OPERATION_HANDLE_ID, + OPERATION_ID, + OPTIMIZED_PLAN_COLUMNS, + OPTIMIZER_CLASS_NAME, + OPTIONS, + OP_ID, + OP_TYPE, + ORIGINAL_DISK_SIZE, + ORIGINAL_MEMORY_SIZE, + OS_ARCH, + OS_NAME, + OS_VERSION, + OUTPUT, + OUTPUT_BUFFER, + OVERHEAD_MEMORY_SIZE, + PAGE_SIZE, + PARENT_STAGES, + PARSE_MODE, + PARTITIONED_FILE_READER, + PARTITIONER, + PARTITION_ID, + PARTITION_IDS, + PARTITION_SIZE, + PARTITION_SPECIFICATION, + PARTITION_SPECS, + PATH, + PATHS, + PEER, + PENDING_TIMES, + PERCENT, + PIPELINE_STAGE_UID, + PLUGIN_NAME, + POD_ID, + POD_NAME, + POD_NAMESPACE, + POD_PHASE, + POD_STATE, + POINT_OF_CENTER, + POLICY, + POOL_NAME, + PORT, + PORT2, + POST_SCAN_FILTERS, + PREDICATE, + PREDICATES, + PREFERRED_SERVICE_NAME, + PREFIX, + PRETTY_ID_STRING, + PRINCIPAL, + PROCESS, + PROCESSING_TIME, + PRODUCER_ID, + PROPERTY_NAME, + PROPORTIONAL, + PROTOCOL_VERSION, + PROVIDER, + PUSHED_FILTERS, + PUSH_MERGED_LOCAL_BLOCKS_SIZE, + PVC_METADATA_NAME, + PYTHON_EXEC, + PYTHON_PACKAGES, + PYTHON_VERSION, + PYTHON_WORKER_CHANNEL_IS_BLOCKING_MODE, + PYTHON_WORKER_CHANNEL_IS_CONNECTED, + PYTHON_WORKER_HAS_INPUTS, + PYTHON_WORKER_IDLE_TIMEOUT, + PYTHON_WORKER_IS_ALIVE, + PYTHON_WORKER_MODULE, + PYTHON_WORKER_RESPONSE, + PYTHON_WORKER_SELECTION_KEY_INTERESTS, + PYTHON_WORKER_SELECTION_KEY_IS_VALID, + PYTHON_WORKER_SELECTOR_IS_OPEN, + QUANTILES, + QUERY_CACHE_VALUE, + QUERY_HINT, + QUERY_ID, + QUERY_PLAN, + QUERY_PLAN_COMPARISON, + QUERY_PLAN_LENGTH_ACTUAL, + QUERY_PLAN_LENGTH_MAX, + QUERY_RUN_ID, + RANGE, + RATE_LIMIT, + RATIO, + RDD, + RDD_CHECKPOINT_DIR, + RDD_DEBUG_STRING, + RDD_DESCRIPTION, + RDD_ID, + READ_LIMIT, + REASON, + REATTACHABLE, + RECEIVED_BLOCK_INFO, + RECEIVED_BLOCK_TRACKER_LOG_EVENT, + RECEIVER_ID, + RECEIVER_IDS, + RECORDS, + RECOVERY_STATE, + RECURSIVE_DEPTH, + REDACTED_STATEMENT, + REDUCE_ID, + REGEX, + REGISTERED_EXECUTOR_FILE, + REGISTER_MERGE_RESULTS, + RELATION_NAME, + RELATION_OUTPUT, + RELATIVE_TOLERANCE, + RELEASED_LOCKS, + REMAINING_PARTITIONS, + REMOTE_ADDRESS, + REMOTE_BLOCKS_SIZE, + REMOVE_FROM_MASTER, + REPORT_DETAILS, + REQUESTER_SIZE, + REQUEST_EXECUTORS, + REQUEST_ID, + RESOURCE, + RESOURCE_NAME, + RESOURCE_PROFILE_ID, + RESOURCE_PROFILE_IDS, + RESOURCE_PROFILE_TO_TOTAL_EXECS, + RESPONSE_BODY_SIZE, + RESTART_TIME, + RESULT, + RESULT_SIZE_BYTES, + RESULT_SIZE_BYTES_MAX, + RETRY_INTERVAL, + RETRY_WAIT_TIME, + RIGHT_EXPR, + RIGHT_LOGICAL_PLAN_STATS_SIZE_IN_BYTES, + RMSE, + ROCKS_DB_FILE_MAPPING, + ROCKS_DB_LOG_LEVEL, + ROCKS_DB_LOG_MESSAGE, + RPC_ADDRESS, + RPC_ENDPOINT_REF, + RPC_MESSAGE_CAPACITY, + RPC_SSL_ENABLED, + RULE_EXECUTOR_NAME, + RULE_NAME, + RUN_ID, + RUN_ID_STRING, + SCALA_VERSION, + SCALING_DOWN_RATIO, + SCALING_UP_RATIO, + SCHEDULER_POOL_NAME, + SCHEDULING_MODE, + SCHEMA, + SCHEMA2, + SERVER_NAME, + SERVICE_NAME, + SERVLET_CONTEXT_HANDLER_PATH, + SESSION_HANDLE, + SESSION_HOLD_INFO, + SESSION_ID, + SESSION_KEY, + SET_CLIENT_INFO_REQUEST, + SHARD_ID, + SHORTER_SERVICE_NAME, + SHORT_USER_NAME, + SHUFFLE_BLOCK_INFO, + SHUFFLE_DB_BACKEND_KEY, + SHUFFLE_DB_BACKEND_NAME, + SHUFFLE_ID, + SHUFFLE_IDS, + SHUFFLE_MERGE_ID, + SHUFFLE_MERGE_RECOVERY_FILE, + SHUFFLE_SERVICE_CONF_OVERLAY_URL, + SHUFFLE_SERVICE_METRICS_NAMESPACE, + SHUFFLE_SERVICE_NAME, + SIGMAS_LENGTH, + SIGNAL, + SINK, + SIZE, + SLEEP_TIME, + SLIDE_DURATION, + SMALLEST_CLUSTER_INDEX, + SNAPSHOT_EVENT, + SNAPSHOT_EVENT_TIME_DELTA, + SNAPSHOT_EVENT_VERSION_DELTA, + SNAPSHOT_VERSION, + SOCKET_ADDRESS, + SOURCE, + SOURCE_PATH, + SPARK_BRANCH, + SPARK_BUILD_DATE, + SPARK_BUILD_USER, + SPARK_DATA_STREAM, + SPARK_PLAN_ID, + SPARK_REPO_URL, + SPARK_REVISION, + SPARK_VERSION, + SPILL_RECORDS_SIZE, + SPILL_RECORDS_SIZE_THRESHOLD, + SPILL_TIMES, + SQL_TEXT, + SRC_PATH, + STAGE, + STAGES, + STAGE_ATTEMPT, + STAGE_ATTEMPT_ID, + STAGE_ID, + STAGE_NAME, + STAMP, + START_INDEX, + START_TIME, + STATEMENT_ID, + STATE_NAME, + STATE_STORE_COORDINATOR, + STATE_STORE_ID, + STATE_STORE_PROVIDER, + STATE_STORE_PROVIDER_ID, + STATE_STORE_PROVIDER_IDS, + STATE_STORE_STATE, + STATE_STORE_VERSION, + STATS, + STATUS, + STDERR, + STOP_SITE_SHORT_FORM, + STORAGE_LEVEL, + STORAGE_LEVEL_DESERIALIZED, + STORAGE_LEVEL_REPLICATION, + STORAGE_MEMORY_SIZE, + STORE_ID, + STRATEGY, + STREAMING_CONTEXT, + STREAMING_DATA_SOURCE_DESCRIPTION, + STREAMING_DATA_SOURCE_NAME, + STREAMING_OFFSETS_END, + STREAMING_OFFSETS_START, + STREAMING_QUERY_PROGRESS, + STREAMING_SOURCE, + STREAMING_TABLE, + STREAMING_WRITE, + STREAM_CHUNK_ID, + STREAM_ID, + STREAM_NAME, + SUBMISSION_ID, + SUBSAMPLING_RATE, + SUB_QUERY, + TABLE_NAME, + TABLE_TYPE, + TABLE_TYPES, + TAG, + TARGET_NUM_EXECUTOR, + TARGET_NUM_EXECUTOR_DELTA, + TARGET_PATH, + TARGET_SIZE, + TASK_ATTEMPT_ID, + TASK_ID, + TASK_INDEX, + TASK_LOCALITY, + TASK_NAME, + TASK_REQUIREMENTS, + TASK_RESOURCES, + TASK_RESOURCE_ASSIGNMENTS, + TASK_SET_MANAGER, + TASK_SET_NAME, + TASK_STATE, + TEMP_FILE, + TEMP_OUTPUT_PATH, + TEMP_PATH, + TEST_SIZE, + THREAD, + THREAD_ID, + THREAD_NAME, + THREAD_POOL_KEEPALIVE_TIME, + THREAD_POOL_SIZE, + THREAD_POOL_WAIT_QUEUE_SIZE, + THRESHOLD, + THRESH_TIME, + TIME, + TIMEOUT, + TIMER, + TIMESTAMP, + TIME_UNITS, + TIP, + TOKEN, + TOKEN_KIND, + TOKEN_REGEX, + TOKEN_RENEWER, + TOPIC, + TOPIC_PARTITION, + TOPIC_PARTITIONS, + TOPIC_PARTITION_OFFSET, + TOPIC_PARTITION_OFFSET_RANGE, + TOTAL, + TOTAL_EFFECTIVE_TIME, + TOTAL_SIZE, + TOTAL_TIME, + TOTAL_TIME_READ, + TO_TIME, + TRAINING_SIZE, + TRAIN_VALIDATION_SPLIT_METRIC, + TRAIN_VALIDATION_SPLIT_METRICS, + TRANSFER_TYPE, + TREE_NODE, + TRIGGER_INTERVAL, + UI_ACLS, + UI_FILTER, + UI_FILTER_PARAMS, + UI_PROXY_BASE, + UNKNOWN_PARAM, + UNSUPPORTED_EXPR, + UNSUPPORTED_HINT_REASON, + UNTIL_OFFSET, + UPPER_BOUND, + URI, + URIS, + URL, + URL2, + URLS, + USER_ID, + USER_NAME, + UUID, + VALUE, + VERSIONS_TO_DELETE, + VERSION_NUM, + VIEW_ACLS, + VIEW_ACLS_GROUPS, + VIRTUAL_CORES, + VOCAB_SIZE, + WAIT_RESULT_TIME, + WAIT_SEND_TIME, + WATERMARK_CONSTRAINT, + WEB_URL, + WEIGHT, + WORKER, + WORKER_HOST, + WORKER_ID, + WORKER_PORT, + WORKER_URL, + WRITE_AHEAD_LOG_INFO, + WRITE_AHEAD_LOG_RECORD_HANDLE, + WRITE_JOB_UUID, + XML_SCHEDULING_MODE, + XSD_PATH, + YARN_RESOURCE, + YOUNG_GENERATION_GC, + ZERO_TIME +} diff --git a/common/utils/src/main/java/org/apache/spark/internal/MDC.java b/common/utils/src/main/java/org/apache/spark/internal/MDC.java new file mode 100644 index 0000000000000..341967812c84b --- /dev/null +++ b/common/utils/src/main/java/org/apache/spark/internal/MDC.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.internal; + +/** + * Mapped Diagnostic Context (MDC) that will be used in log messages. + * The values of the MDC will be inline in the log message, while the key-value pairs will be + * part of the ThreadContext. + */ +public record MDC(LogKey key, Object value) { + public static MDC of(LogKey key, Object value) { + return new MDC(key, value); + } +} diff --git a/common/utils/src/main/java/org/apache/spark/internal/SparkLogger.java b/common/utils/src/main/java/org/apache/spark/internal/SparkLogger.java index 8c210a4fab3c3..84d6d7cf4238c 100644 --- a/common/utils/src/main/java/org/apache/spark/internal/SparkLogger.java +++ b/common/utils/src/main/java/org/apache/spark/internal/SparkLogger.java @@ -18,6 +18,7 @@ package org.apache.spark.internal; import java.util.HashMap; +import java.util.Locale; import java.util.Map; import java.util.function.Consumer; @@ -51,7 +52,7 @@ * * import org.apache.spark.internal.LogKeys; * import org.apache.spark.internal.MDC; - * logger.error("Unable to delete file for partition {}", MDC.of(LogKeys.PARTITION_ID$.MODULE$, i)); + * logger.error("Unable to delete file for partition {}", MDC.of(LogKeys.PARTITION_ID, i)); *

* * Constant String Messages: @@ -65,8 +66,10 @@ * you can define `custom LogKey` and use it in `java` code as follows: *

* - * // To add a `custom LogKey`, implement `LogKey` - * public static class CUSTOM_LOG_KEY implements LogKey { } + * // Add a `CustomLogKeys`, implement `LogKey` + * public enum CustomLogKeys implements LogKey { + * CUSTOM_LOG_KEY + * } * import org.apache.spark.internal.MDC; * logger.error("Unable to delete key {} for cache", MDC.of(CUSTOM_LOG_KEY, "key")); */ @@ -222,8 +225,8 @@ private void withLogContext( for (int index = 0; index < mdcs.length; index++) { MDC mdc = mdcs[index]; String value = (mdc.value() != null) ? mdc.value().toString() : null; - if (Logging$.MODULE$.isStructuredLoggingEnabled()) { - context.put(mdc.key().name(), value); + if (SparkLoggerFactory.isStructuredLoggingEnabled()) { + context.put(mdc.key().name().toLowerCase(Locale.ROOT), value); } args[index] = value; } diff --git a/common/utils/src/main/java/org/apache/spark/internal/SparkLoggerFactory.java b/common/utils/src/main/java/org/apache/spark/internal/SparkLoggerFactory.java index a59c007362419..f5be570fa5b39 100644 --- a/common/utils/src/main/java/org/apache/spark/internal/SparkLoggerFactory.java +++ b/common/utils/src/main/java/org/apache/spark/internal/SparkLoggerFactory.java @@ -23,6 +23,20 @@ public class SparkLoggerFactory { + private static volatile boolean structuredLoggingEnabled = false; + + public static void enableStructuredLogging() { + structuredLoggingEnabled = true; + } + + public static void disableStructuredLogging() { + structuredLoggingEnabled = false; + } + + public static boolean isStructuredLoggingEnabled() { + return structuredLoggingEnabled; + } + public static SparkLogger getLogger(String name) { return new SparkLogger(LoggerFactory.getLogger(name)); } diff --git a/common/utils/src/main/java/org/apache/spark/network/util/JavaUtils.java b/common/utils/src/main/java/org/apache/spark/network/util/JavaUtils.java index 376b577adcc2f..f1dfa9106e3e9 100644 --- a/common/utils/src/main/java/org/apache/spark/network/util/JavaUtils.java +++ b/common/utils/src/main/java/org/apache/spark/network/util/JavaUtils.java @@ -245,7 +245,7 @@ public static void deleteRecursively(File file, FilenameFilter filter) return; } catch (IOException e) { logger.warn("Attempt to delete using native Unix OS command failed for path = {}. " + - "Falling back to Java IO way", e, MDC.of(LogKeys.PATH$.MODULE$, file.getAbsolutePath())); + "Falling back to Java IO way", e, MDC.of(LogKeys.PATH, file.getAbsolutePath())); } } @@ -543,7 +543,7 @@ public static File createDirectory(String root, String namePrefix) throws IOExce dir = new File(root, namePrefix + "-" + UUID.randomUUID()); Files.createDirectories(dir.toPath()); } catch (IOException | SecurityException e) { - logger.error("Failed to create directory {}", e, MDC.of(LogKeys.PATH$.MODULE$, dir)); + logger.error("Failed to create directory {}", e, MDC.of(LogKeys.PATH, dir)); dir = null; } } diff --git a/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala b/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala deleted file mode 100644 index a35af2d80c876..0000000000000 --- a/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala +++ /dev/null @@ -1,925 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.internal - -import java.util.Locale - -/** - * All structured logging `keys` used in `MDC` must be extends `LogKey` - *

- * - * `LogKey`s serve as identifiers for mapped diagnostic contexts (MDC) within logs. - * Follow these guidelines when adding a new LogKey: - *

    - *
  • - * Define all structured logging keys in `LogKey.scala`, and sort them alphabetically for - * ease of search. - *
  • - *
  • - * Use `UPPER_SNAKE_CASE` for key names. - *
  • - *
  • - * Key names should be both simple and broad, yet include specific identifiers like `STAGE_ID`, - * `TASK_ID`, and `JOB_ID` when needed for clarity. For instance, use `MAX_ATTEMPTS` as a - * general key instead of creating separate keys for each scenario such as - * `EXECUTOR_STATE_SYNC_MAX_ATTEMPTS` and `MAX_TASK_FAILURES`. - * This balances simplicity with the detail needed for effective logging. - *
  • - *
  • - * Use abbreviations in names if they are widely understood, - * such as `APP_ID` for APPLICATION_ID, and `K8S` for KUBERNETES. - *
  • - *
  • - * For time-related keys, use milliseconds as the unit of time. - *
  • - *
- */ -trait LogKey { - private lazy val _name: String = getClass.getSimpleName.stripSuffix("$").toLowerCase(Locale.ROOT) - def name: String = _name -} - -/** - * Various keys used for mapped diagnostic contexts(MDC) in logging. All structured logging keys - * should be defined here for standardization. - */ -private[spark] object LogKeys { - case object ACCUMULATOR_ID extends LogKey - case object ACL_ENABLED extends LogKey - case object ACTUAL_NUM_FILES extends LogKey - case object ACTUAL_PARTITION_COLUMN extends LogKey - case object ADDED_JARS extends LogKey - case object ADMIN_ACLS extends LogKey - case object ADMIN_ACL_GROUPS extends LogKey - case object ADVISORY_TARGET_SIZE extends LogKey - case object AGGREGATE_FUNCTIONS extends LogKey - case object ALIGNED_FROM_TIME extends LogKey - case object ALIGNED_TO_TIME extends LogKey - case object ALPHA extends LogKey - case object ANALYSIS_ERROR extends LogKey - case object ANTLR_DFA_CACHE_DELTA extends LogKey - case object ANTLR_DFA_CACHE_SIZE extends LogKey - case object APP_ATTEMPT_ID extends LogKey - case object APP_ATTEMPT_SHUFFLE_MERGE_ID extends LogKey - case object APP_DESC extends LogKey - case object APP_EXECUTOR_ID extends LogKey - case object APP_ID extends LogKey - case object APP_NAME extends LogKey - case object APP_STATE extends LogKey - case object ARCHIVE_NAME extends LogKey - case object ARGS extends LogKey - case object ARTIFACTS extends LogKey - case object ARTIFACT_ID extends LogKey - case object ATTRIBUTE_MAP extends LogKey - case object AUTH_ENABLED extends LogKey - case object AVG_BATCH_PROC_TIME extends LogKey - case object BACKUP_FILE extends LogKey - case object BARRIER_EPOCH extends LogKey - case object BARRIER_ID extends LogKey - case object BATCH_ID extends LogKey - case object BATCH_NAME extends LogKey - case object BATCH_TIMES extends LogKey - case object BATCH_TIMESTAMP extends LogKey - case object BATCH_WRITE extends LogKey - case object BIND_ADDRESS extends LogKey - case object BLOCK_GENERATOR_STATUS extends LogKey - case object BLOCK_ID extends LogKey - case object BLOCK_IDS extends LogKey - case object BLOCK_MANAGER_ID extends LogKey - case object BLOCK_MANAGER_IDS extends LogKey - case object BLOCK_TYPE extends LogKey - case object BOOT extends LogKey - case object BOOTSTRAP_TIME extends LogKey - case object BOOT_TIME extends LogKey - case object BROADCAST extends LogKey - case object BROADCAST_ID extends LogKey - case object BROADCAST_OUTPUT_STATUS_SIZE extends LogKey - case object BUCKET extends LogKey - case object BYTECODE_SIZE extends LogKey - case object BYTE_BUFFER extends LogKey - case object BYTE_SIZE extends LogKey - case object CACHED_TABLE_PARTITION_METADATA_SIZE extends LogKey - case object CACHE_AUTO_REMOVED_SIZE extends LogKey - case object CACHE_SIZE extends LogKey - case object CACHE_UNTIL_HIGHEST_CONSUMED_SIZE extends LogKey - case object CACHE_UNTIL_LAST_PRODUCED_SIZE extends LogKey - case object CALL_SITE_LONG_FORM extends LogKey - case object CALL_SITE_SHORT_FORM extends LogKey - case object CANCEL_FUTURE_JOBS extends LogKey - case object CATALOG_NAME extends LogKey - case object CATEGORICAL_FEATURES extends LogKey - case object CHECKPOINT_FILE extends LogKey - case object CHECKPOINT_INTERVAL extends LogKey - case object CHECKPOINT_LOCATION extends LogKey - case object CHECKPOINT_PATH extends LogKey - case object CHECKPOINT_ROOT extends LogKey - case object CHECKPOINT_TIME extends LogKey - case object CHOSEN_WATERMARK extends LogKey - case object CLASSIFIER extends LogKey - case object CLASS_LOADER extends LogKey - case object CLASS_NAME extends LogKey - case object CLASS_PATH extends LogKey - case object CLASS_PATHS extends LogKey - case object CLAUSES extends LogKey - case object CLEANUP_LOCAL_DIRS extends LogKey - case object CLUSTER_CENTROIDS extends LogKey - case object CLUSTER_ID extends LogKey - case object CLUSTER_LABEL extends LogKey - case object CLUSTER_LEVEL extends LogKey - case object CLUSTER_WEIGHT extends LogKey - case object CODE extends LogKey - case object CODEC_LEVEL extends LogKey - case object CODEC_NAME extends LogKey - case object CODEGEN_STAGE_ID extends LogKey - case object COLUMN_DATA_TYPE_SOURCE extends LogKey - case object COLUMN_DATA_TYPE_TARGET extends LogKey - case object COLUMN_DEFAULT_VALUE extends LogKey - case object COLUMN_NAME extends LogKey - case object COLUMN_NAMES extends LogKey - case object COMMAND extends LogKey - case object COMMAND_OUTPUT extends LogKey - case object COMMITTED_VERSION extends LogKey - case object COMPACT_INTERVAL extends LogKey - case object COMPONENT extends LogKey - case object COMPUTE extends LogKey - case object CONFIG extends LogKey - case object CONFIG2 extends LogKey - case object CONFIG3 extends LogKey - case object CONFIG4 extends LogKey - case object CONFIG5 extends LogKey - case object CONFIG_DEPRECATION_MESSAGE extends LogKey - case object CONFIG_KEY_UPDATED extends LogKey - case object CONFIG_VERSION extends LogKey - case object CONSUMER extends LogKey - case object CONTAINER extends LogKey - case object CONTAINER_ID extends LogKey - case object CONTAINER_STATE extends LogKey - case object CONTEXT extends LogKey - case object COST extends LogKey - case object COUNT extends LogKey - case object CREATED_POOL_NAME extends LogKey - case object CREATION_SITE extends LogKey - case object CREDENTIALS_RENEWAL_INTERVAL_RATIO extends LogKey - case object CROSS_VALIDATION_METRIC extends LogKey - case object CROSS_VALIDATION_METRICS extends LogKey - case object CSV_HEADER_COLUMN_NAME extends LogKey - case object CSV_HEADER_COLUMN_NAMES extends LogKey - case object CSV_HEADER_LENGTH extends LogKey - case object CSV_SCHEMA_FIELD_NAME extends LogKey - case object CSV_SCHEMA_FIELD_NAMES extends LogKey - case object CSV_SOURCE extends LogKey - case object CURRENT_BATCH_ID extends LogKey - case object CURRENT_DISK_SIZE extends LogKey - case object CURRENT_FILE extends LogKey - case object CURRENT_MEMORY_SIZE extends LogKey - case object CURRENT_PATH extends LogKey - case object CURRENT_TIME extends LogKey - case object DATA extends LogKey - case object DATABASE_NAME extends LogKey - case object DATAFRAME_CACHE_ENTRY extends LogKey - case object DATAFRAME_ID extends LogKey - case object DATA_FILE extends LogKey - case object DATA_SOURCE extends LogKey - case object DATA_SOURCES extends LogKey - case object DEFAULT_COMPACT_INTERVAL extends LogKey - case object DEFAULT_ISOLATION_LEVEL extends LogKey - case object DEFAULT_NAME extends LogKey - case object DEFAULT_VALUE extends LogKey - case object DELAY extends LogKey - case object DELEGATE extends LogKey - case object DELTA extends LogKey - case object DEPRECATED_KEY extends LogKey - case object DERIVATIVE extends LogKey - case object DESCRIPTION extends LogKey - case object DESIRED_NUM_PARTITIONS extends LogKey - case object DESIRED_TREE_DEPTH extends LogKey - case object DESTINATION_PATH extends LogKey - case object DFS_FILE extends LogKey - case object DIFF_DELTA extends LogKey - case object DIVISIBLE_CLUSTER_INDICES_SIZE extends LogKey - case object DRIVER_ID extends LogKey - case object DRIVER_JVM_MEMORY extends LogKey - case object DRIVER_MEMORY_SIZE extends LogKey - case object DRIVER_STATE extends LogKey - case object DROPPED_PARTITIONS extends LogKey - case object DSTREAM extends LogKey - case object DURATION extends LogKey - case object EARLIEST_LOADED_VERSION extends LogKey - case object EFFECTIVE_STORAGE_LEVEL extends LogKey - case object ELAPSED_TIME extends LogKey - case object ENCODING extends LogKey - case object ENDPOINT_NAME extends LogKey - case object END_INDEX extends LogKey - case object END_POINT extends LogKey - case object END_VERSION extends LogKey - case object ENGINE extends LogKey - case object EPOCH extends LogKey - case object ERROR extends LogKey - case object ESTIMATOR_PARAM_MAP extends LogKey - case object EVALUATED_FILTERS extends LogKey - case object EVENT extends LogKey - case object EVENT_LOG_DESTINATION extends LogKey - case object EVENT_LOOP extends LogKey - case object EVENT_NAME extends LogKey - case object EVENT_QUEUE extends LogKey - case object EXCEPTION extends LogKey - case object EXECUTE_INFO extends LogKey - case object EXECUTE_KEY extends LogKey - case object EXECUTION_MEMORY_SIZE extends LogKey - case object EXECUTION_PLAN_LEAVES extends LogKey - case object EXECUTOR_BACKEND extends LogKey - case object EXECUTOR_ENVS extends LogKey - case object EXECUTOR_ENV_REGEX extends LogKey - case object EXECUTOR_ID extends LogKey - case object EXECUTOR_IDS extends LogKey - case object EXECUTOR_LAUNCH_COMMANDS extends LogKey - case object EXECUTOR_MEMORY_OFFHEAP extends LogKey - case object EXECUTOR_MEMORY_OVERHEAD_SIZE extends LogKey - case object EXECUTOR_MEMORY_SIZE extends LogKey - case object EXECUTOR_RESOURCES extends LogKey - case object EXECUTOR_SHUFFLE_INFO extends LogKey - case object EXECUTOR_STATE extends LogKey - case object EXECUTOR_TIMEOUT extends LogKey - case object EXECUTOR_USER_CLASS_PATH_FIRST extends LogKey - case object EXEC_AMOUNT extends LogKey - case object EXISTING_FILE extends LogKey - case object EXISTING_PATH extends LogKey - case object EXIT_CODE extends LogKey - case object EXPECTED_NUM_FILES extends LogKey - case object EXPECTED_PARTITION_COLUMN extends LogKey - case object EXPIRY_TIMESTAMP extends LogKey - case object EXPR extends LogKey - case object EXPR_TERMS extends LogKey - case object EXTENDED_EXPLAIN_GENERATOR extends LogKey - case object FAILED_STAGE extends LogKey - case object FAILED_STAGE_NAME extends LogKey - case object FAILURES extends LogKey - case object FALLBACK_VERSION extends LogKey - case object FEATURE_COLUMN extends LogKey - case object FEATURE_DIMENSION extends LogKey - case object FEATURE_NAME extends LogKey - case object FETCH_SIZE extends LogKey - case object FIELD_NAME extends LogKey - case object FIELD_TYPE extends LogKey - case object FILES extends LogKey - case object FILE_ABSOLUTE_PATH extends LogKey - case object FILE_END_OFFSET extends LogKey - case object FILE_FORMAT extends LogKey - case object FILE_FORMAT2 extends LogKey - case object FILE_LENGTH_XATTR extends LogKey - case object FILE_MODIFICATION_TIME extends LogKey - case object FILE_NAME extends LogKey - case object FILE_NAME2 extends LogKey - case object FILE_NAME3 extends LogKey - case object FILE_NAMES extends LogKey - case object FILE_START_OFFSET extends LogKey - case object FILE_SYSTEM extends LogKey - case object FILE_VERSION extends LogKey - case object FILTER extends LogKey - case object FINAL_CONTEXT extends LogKey - case object FINAL_OUTPUT_PATH extends LogKey - case object FINAL_PATH extends LogKey - case object FINISH_TIME extends LogKey - case object FINISH_TRIGGER_DURATION extends LogKey - case object FLOW_NAME extends LogKey - case object FREE_MEMORY_SIZE extends LogKey - case object FROM_OFFSET extends LogKey - case object FROM_TIME extends LogKey - case object FS_DATA_OUTPUT_STREAM extends LogKey - case object FUNCTION_NAME extends LogKey - case object FUNCTION_PARAM extends LogKey - case object GLOBAL_INIT_FILE extends LogKey - case object GLOBAL_WATERMARK extends LogKey - case object GROUP_BY_EXPRS extends LogKey - case object GROUP_ID extends LogKey - case object HADOOP_VERSION extends LogKey - case object HASH_JOIN_KEYS extends LogKey - case object HASH_MAP_SIZE extends LogKey - case object HEARTBEAT extends LogKey - case object HEARTBEAT_INTERVAL extends LogKey - case object HISTORY_DIR extends LogKey - case object HIVE_CLIENT_VERSION extends LogKey - case object HIVE_METASTORE_VERSION extends LogKey - case object HIVE_OPERATION_STATE extends LogKey - case object HIVE_OPERATION_TYPE extends LogKey - case object HOST extends LogKey - case object HOSTS extends LogKey - case object HOST_LOCAL_BLOCKS_SIZE extends LogKey - case object HOST_PORT extends LogKey - case object HOST_PORT2 extends LogKey - case object HUGE_METHOD_LIMIT extends LogKey - case object HYBRID_STORE_DISK_BACKEND extends LogKey - case object IDENTIFIER extends LogKey - case object INCOMPATIBLE_TYPES extends LogKey - case object INDEX extends LogKey - case object INDEX_FILE extends LogKey - case object INDEX_NAME extends LogKey - case object INFERENCE_MODE extends LogKey - case object INIT extends LogKey - case object INITIAL_CAPACITY extends LogKey - case object INITIAL_HEARTBEAT_INTERVAL extends LogKey - case object INIT_MODE extends LogKey - case object INIT_TIME extends LogKey - case object INPUT extends LogKey - case object INPUT_SPLIT extends LogKey - case object INTEGRAL extends LogKey - case object INTERRUPT_THREAD extends LogKey - case object INTERVAL extends LogKey - case object INVALID_PARAMS extends LogKey - case object ISOLATION_LEVEL extends LogKey - case object ISSUE_DATE extends LogKey - case object IS_NETWORK_REQUEST_DONE extends LogKey - case object JAR_ENTRY extends LogKey - case object JAR_MESSAGE extends LogKey - case object JAR_URL extends LogKey - case object JAVA_VERSION extends LogKey - case object JAVA_VM_NAME extends LogKey - case object JOB_ID extends LogKey - case object JOIN_CONDITION extends LogKey - case object JOIN_CONDITION_SUB_EXPR extends LogKey - case object JOIN_TYPE extends LogKey - case object K8S_CONTEXT extends LogKey - case object KEY extends LogKey - case object KEY2 extends LogKey - case object KEYTAB extends LogKey - case object KEYTAB_FILE extends LogKey - case object KILL_EXECUTORS extends LogKey - case object KINESIS_REASON extends LogKey - case object LABEL_COLUMN extends LogKey - case object LARGEST_CLUSTER_INDEX extends LogKey - case object LAST_ACCESS_TIME extends LogKey - case object LAST_COMMITTED_CHECKPOINT_ID extends LogKey - case object LAST_COMMIT_BASED_CHECKPOINT_ID extends LogKey - case object LAST_VALID_TIME extends LogKey - case object LATEST_BATCH_ID extends LogKey - case object LATEST_COMMITTED_BATCH_ID extends LogKey - case object LATEST_SHUFFLE_MERGE_ID extends LogKey - case object LEARNING_RATE extends LogKey - case object LEFT_EXPR extends LogKey - case object LEFT_LOGICAL_PLAN_STATS_SIZE_IN_BYTES extends LogKey - case object LINE extends LogKey - case object LINEAGE extends LogKey - case object LINE_NUM extends LogKey - case object LISTENER extends LogKey - case object LOADED_CHECKPOINT_ID extends LogKey - case object LOADED_VERSION extends LogKey - case object LOAD_FACTOR extends LogKey - case object LOAD_TIME extends LogKey - case object LOCALE extends LogKey - case object LOCAL_BLOCKS_SIZE extends LogKey - case object LOCAL_SCRATCH_DIR extends LogKey - case object LOCATION extends LogKey - case object LOGICAL_PLAN extends LogKey - case object LOGICAL_PLAN_COLUMNS extends LogKey - case object LOGICAL_PLAN_LEAVES extends LogKey - case object LOG_ID extends LogKey - case object LOG_LEVEL extends LogKey - case object LOG_OFFSET extends LogKey - case object LOG_TYPE extends LogKey - case object LOSSES extends LogKey - case object LOWER_BOUND extends LogKey - case object MALFORMATTED_STRING extends LogKey - case object MAP_ID extends LogKey - case object MASTER_URL extends LogKey - case object MAX_ATTEMPTS extends LogKey - case object MAX_CACHE_UNTIL_HIGHEST_CONSUMED_SIZE extends LogKey - case object MAX_CACHE_UNTIL_LAST_PRODUCED_SIZE extends LogKey - case object MAX_CAPACITY extends LogKey - case object MAX_CATEGORIES extends LogKey - case object MAX_EXECUTOR_FAILURES extends LogKey - case object MAX_FILE_VERSION extends LogKey - case object MAX_JVM_METHOD_PARAMS_LENGTH extends LogKey - case object MAX_MEMORY_SIZE extends LogKey - case object MAX_METHOD_CODE_SIZE extends LogKey - case object MAX_NUM_BINS extends LogKey - case object MAX_NUM_CHUNKS extends LogKey - case object MAX_NUM_FILES extends LogKey - case object MAX_NUM_LOG_POLICY extends LogKey - case object MAX_NUM_PARTITIONS extends LogKey - case object MAX_NUM_POSSIBLE_BINS extends LogKey - case object MAX_NUM_ROWS_IN_MEMORY_BUFFER extends LogKey - case object MAX_SEEN_VERSION extends LogKey - case object MAX_SERVICE_NAME_LENGTH extends LogKey - case object MAX_SIZE extends LogKey - case object MAX_SLOTS extends LogKey - case object MAX_SPLIT_BYTES extends LogKey - case object MAX_TABLE_PARTITION_METADATA_SIZE extends LogKey - case object MEMORY_CONSUMER extends LogKey - case object MEMORY_POOL_NAME extends LogKey - case object MEMORY_SIZE extends LogKey - case object MEMORY_THRESHOLD_SIZE extends LogKey - case object MERGE_DIR_NAME extends LogKey - case object MESSAGE extends LogKey - case object METADATA extends LogKey - case object METADATA_DIRECTORY extends LogKey - case object METADATA_JSON extends LogKey - case object META_FILE extends LogKey - case object METHOD_NAME extends LogKey - case object METHOD_PARAM_TYPES extends LogKey - case object METRICS_JSON extends LogKey - case object METRIC_NAME extends LogKey - case object MINI_BATCH_FRACTION extends LogKey - case object MIN_COMPACTION_BATCH_ID extends LogKey - case object MIN_NUM_FREQUENT_PATTERN extends LogKey - case object MIN_POINT_PER_CLUSTER extends LogKey - case object MIN_RATE extends LogKey - case object MIN_SEEN_VERSION extends LogKey - case object MIN_SHARE extends LogKey - case object MIN_SIZE extends LogKey - case object MIN_TIME extends LogKey - case object MIN_VERSIONS_TO_DELETE extends LogKey - case object MIN_VERSION_NUM extends LogKey - case object MISSING_PARENT_STAGES extends LogKey - case object MODEL_WEIGHTS extends LogKey - case object MODIFY_ACLS extends LogKey - case object MODIFY_ACLS_GROUPS extends LogKey - case object MODULE_NAME extends LogKey - case object NAME extends LogKey - case object NAMESPACE extends LogKey - case object NETWORK_IF extends LogKey - case object NEW_FEATURE_COLUMN_NAME extends LogKey - case object NEW_LABEL_COLUMN_NAME extends LogKey - case object NEW_PATH extends LogKey - case object NEW_RDD_ID extends LogKey - case object NEW_STATE extends LogKey - case object NEW_VALUE extends LogKey - case object NEXT_RENEWAL_TIME extends LogKey - case object NODES extends LogKey - case object NODE_LOCATION extends LogKey - case object NON_BUILT_IN_CONNECTORS extends LogKey - case object NORM extends LogKey - case object NUM_ADDED_PARTITIONS extends LogKey - case object NUM_APPS extends LogKey - case object NUM_ATTEMPT extends LogKey - case object NUM_BATCHES extends LogKey - case object NUM_BIN extends LogKey - case object NUM_BLOCKS extends LogKey - case object NUM_BLOCK_IDS extends LogKey - case object NUM_BROADCAST_BLOCK extends LogKey - case object NUM_BYTES extends LogKey - case object NUM_BYTES_CURRENT extends LogKey - case object NUM_BYTES_EVICTED extends LogKey - case object NUM_BYTES_MAX extends LogKey - case object NUM_BYTES_TO_FREE extends LogKey - case object NUM_BYTES_TO_WARN extends LogKey - case object NUM_BYTES_USED extends LogKey - case object NUM_CATEGORIES extends LogKey - case object NUM_CHECKSUM_FILE extends LogKey - case object NUM_CHUNKS extends LogKey - case object NUM_CLASSES extends LogKey - case object NUM_COEFFICIENTS extends LogKey - case object NUM_COLUMNS extends LogKey - case object NUM_CONCURRENT_WRITER extends LogKey - case object NUM_CORES extends LogKey - case object NUM_DATA_FILE extends LogKey - case object NUM_DATA_FILES extends LogKey - case object NUM_DECOMMISSIONED extends LogKey - case object NUM_DRIVERS extends LogKey - case object NUM_DROPPED_PARTITIONS extends LogKey - case object NUM_EFFECTIVE_RULE_OF_RUNS extends LogKey - case object NUM_ELEMENTS_SPILL_RECORDS extends LogKey - case object NUM_ELEMENTS_SPILL_THRESHOLD extends LogKey - case object NUM_EVENTS extends LogKey - case object NUM_EXAMPLES extends LogKey - case object NUM_EXECUTORS extends LogKey - case object NUM_EXECUTORS_EXITED extends LogKey - case object NUM_EXECUTORS_KILLED extends LogKey - case object NUM_EXECUTOR_CORES extends LogKey - case object NUM_EXECUTOR_CORES_REMAINING extends LogKey - case object NUM_EXECUTOR_CORES_TOTAL extends LogKey - case object NUM_EXECUTOR_DESIRED extends LogKey - case object NUM_EXECUTOR_LAUNCH extends LogKey - case object NUM_EXECUTOR_TARGET extends LogKey - case object NUM_FAILURES extends LogKey - case object NUM_FEATURES extends LogKey - case object NUM_FILES extends LogKey - case object NUM_FILES_COPIED extends LogKey - case object NUM_FILES_FAILED_TO_DELETE extends LogKey - case object NUM_FILES_REUSED extends LogKey - case object NUM_FREQUENT_ITEMS extends LogKey - case object NUM_HOST_LOCAL_BLOCKS extends LogKey - case object NUM_INDEX_FILE extends LogKey - case object NUM_INDEX_FILES extends LogKey - case object NUM_ITERATIONS extends LogKey - case object NUM_KAFKA_PULLS extends LogKey - case object NUM_KAFKA_RECORDS_PULLED extends LogKey - case object NUM_LAGGING_STORES extends LogKey - case object NUM_LEADING_SINGULAR_VALUES extends LogKey - case object NUM_LEFT_PARTITION_VALUES extends LogKey - case object NUM_LOADED_ENTRIES extends LogKey - case object NUM_LOCAL_BLOCKS extends LogKey - case object NUM_LOCAL_DIRS extends LogKey - case object NUM_LOCAL_FREQUENT_PATTERN extends LogKey - case object NUM_MERGERS extends LogKey - case object NUM_MERGER_LOCATIONS extends LogKey - case object NUM_META_FILES extends LogKey - case object NUM_NODES extends LogKey - case object NUM_PARTITIONS extends LogKey - case object NUM_PARTITIONS2 extends LogKey - case object NUM_PATHS extends LogKey - case object NUM_PEERS extends LogKey - case object NUM_PEERS_REPLICATED_TO extends LogKey - case object NUM_PEERS_TO_REPLICATE_TO extends LogKey - case object NUM_PENDING_LAUNCH_TASKS extends LogKey - case object NUM_POD extends LogKey - case object NUM_POD_SHARED_SLOT extends LogKey - case object NUM_POD_TARGET extends LogKey - case object NUM_POINT extends LogKey - case object NUM_PREFIXES extends LogKey - case object NUM_PRUNED extends LogKey - case object NUM_PUSH_MERGED_LOCAL_BLOCKS extends LogKey - case object NUM_RECEIVERS extends LogKey - case object NUM_RECORDS_READ extends LogKey - case object NUM_RELEASED_LOCKS extends LogKey - case object NUM_REMAINED extends LogKey - case object NUM_REMOTE_BLOCKS extends LogKey - case object NUM_REMOVED_WORKERS extends LogKey - case object NUM_REPLICAS extends LogKey - case object NUM_REQUESTS extends LogKey - case object NUM_REQUEST_SYNC_TASK extends LogKey - case object NUM_RESOURCE_SLOTS extends LogKey - case object NUM_RETRIES extends LogKey - case object NUM_RETRY extends LogKey - case object NUM_RIGHT_PARTITION_VALUES extends LogKey - case object NUM_ROWS extends LogKey - case object NUM_RULE_OF_RUNS extends LogKey - case object NUM_SEQUENCES extends LogKey - case object NUM_SKIPPED extends LogKey - case object NUM_SLOTS extends LogKey - case object NUM_SPILLS extends LogKey - case object NUM_SPILL_WRITERS extends LogKey - case object NUM_SUB_DIRS extends LogKey - case object NUM_SUCCESSFUL_TASKS extends LogKey - case object NUM_TASKS extends LogKey - case object NUM_TASK_CPUS extends LogKey - case object NUM_TRAIN_WORD extends LogKey - case object NUM_UNFINISHED_DECOMMISSIONED extends LogKey - case object NUM_VERSIONS_RETAIN extends LogKey - case object NUM_WEIGHTED_EXAMPLES extends LogKey - case object NUM_WORKERS extends LogKey - case object OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD extends LogKey - case object OBJECT_ID extends LogKey - case object OFFSET extends LogKey - case object OFFSETS extends LogKey - case object OFFSET_SEQUENCE_METADATA extends LogKey - case object OLD_BLOCK_MANAGER_ID extends LogKey - case object OLD_GENERATION_GC extends LogKey - case object OLD_VALUE extends LogKey - case object OPEN_COST_IN_BYTES extends LogKey - case object OPERATION extends LogKey - case object OPERATION_HANDLE extends LogKey - case object OPERATION_HANDLE_ID extends LogKey - case object OPERATION_ID extends LogKey - case object OPTIMIZED_PLAN_COLUMNS extends LogKey - case object OPTIMIZER_CLASS_NAME extends LogKey - case object OPTIONS extends LogKey - case object OP_ID extends LogKey - case object OP_TYPE extends LogKey - case object ORIGINAL_DISK_SIZE extends LogKey - case object ORIGINAL_MEMORY_SIZE extends LogKey - case object OS_ARCH extends LogKey - case object OS_NAME extends LogKey - case object OS_VERSION extends LogKey - case object OUTPUT extends LogKey - case object OUTPUT_BUFFER extends LogKey - case object OVERHEAD_MEMORY_SIZE extends LogKey - case object PAGE_SIZE extends LogKey - case object PARENT_STAGES extends LogKey - case object PARSE_MODE extends LogKey - case object PARTITIONED_FILE_READER extends LogKey - case object PARTITIONER extends LogKey - case object PARTITION_ID extends LogKey - case object PARTITION_IDS extends LogKey - case object PARTITION_SIZE extends LogKey - case object PARTITION_SPECIFICATION extends LogKey - case object PARTITION_SPECS extends LogKey - case object PATH extends LogKey - case object PATHS extends LogKey - case object PEER extends LogKey - case object PENDING_TIMES extends LogKey - case object PERCENT extends LogKey - case object PIPELINE_STAGE_UID extends LogKey - case object PLUGIN_NAME extends LogKey - case object POD_ID extends LogKey - case object POD_NAME extends LogKey - case object POD_NAMESPACE extends LogKey - case object POD_PHASE extends LogKey - case object POD_STATE extends LogKey - case object POINT_OF_CENTER extends LogKey - case object POLICY extends LogKey - case object POOL_NAME extends LogKey - case object PORT extends LogKey - case object PORT2 extends LogKey - case object POST_SCAN_FILTERS extends LogKey - case object PREDICATE extends LogKey - case object PREDICATES extends LogKey - case object PREFERRED_SERVICE_NAME extends LogKey - case object PREFIX extends LogKey - case object PRETTY_ID_STRING extends LogKey - case object PRINCIPAL extends LogKey - case object PROCESS extends LogKey - case object PROCESSING_TIME extends LogKey - case object PRODUCER_ID extends LogKey - case object PROPERTY_NAME extends LogKey - case object PROPORTIONAL extends LogKey - case object PROTOCOL_VERSION extends LogKey - case object PROVIDER extends LogKey - case object PUSHED_FILTERS extends LogKey - case object PUSH_MERGED_LOCAL_BLOCKS_SIZE extends LogKey - case object PVC_METADATA_NAME extends LogKey - case object PYTHON_EXEC extends LogKey - case object PYTHON_PACKAGES extends LogKey - case object PYTHON_VERSION extends LogKey - case object PYTHON_WORKER_CHANNEL_IS_BLOCKING_MODE extends LogKey - case object PYTHON_WORKER_CHANNEL_IS_CONNECTED extends LogKey - case object PYTHON_WORKER_HAS_INPUTS extends LogKey - case object PYTHON_WORKER_IDLE_TIMEOUT extends LogKey - case object PYTHON_WORKER_IS_ALIVE extends LogKey - case object PYTHON_WORKER_MODULE extends LogKey - case object PYTHON_WORKER_RESPONSE extends LogKey - case object PYTHON_WORKER_SELECTION_KEY_INTERESTS extends LogKey - case object PYTHON_WORKER_SELECTION_KEY_IS_VALID extends LogKey - case object PYTHON_WORKER_SELECTOR_IS_OPEN extends LogKey - case object QUANTILES extends LogKey - case object QUERY_CACHE_VALUE extends LogKey - case object QUERY_HINT extends LogKey - case object QUERY_ID extends LogKey - case object QUERY_PLAN extends LogKey - case object QUERY_PLAN_COMPARISON extends LogKey - case object QUERY_PLAN_LENGTH_ACTUAL extends LogKey - case object QUERY_PLAN_LENGTH_MAX extends LogKey - case object QUERY_RUN_ID extends LogKey - case object RANGE extends LogKey - case object RATE_LIMIT extends LogKey - case object RATIO extends LogKey - case object RDD extends LogKey - case object RDD_CHECKPOINT_DIR extends LogKey - case object RDD_DEBUG_STRING extends LogKey - case object RDD_DESCRIPTION extends LogKey - case object RDD_ID extends LogKey - case object READ_LIMIT extends LogKey - case object REASON extends LogKey - case object REATTACHABLE extends LogKey - case object RECEIVED_BLOCK_INFO extends LogKey - case object RECEIVED_BLOCK_TRACKER_LOG_EVENT extends LogKey - case object RECEIVER_ID extends LogKey - case object RECEIVER_IDS extends LogKey - case object RECORDS extends LogKey - case object RECOVERY_STATE extends LogKey - case object RECURSIVE_DEPTH extends LogKey - case object REDACTED_STATEMENT extends LogKey - case object REDUCE_ID extends LogKey - case object REGEX extends LogKey - case object REGISTERED_EXECUTOR_FILE extends LogKey - case object REGISTER_MERGE_RESULTS extends LogKey - case object RELATION_NAME extends LogKey - case object RELATION_OUTPUT extends LogKey - case object RELATIVE_TOLERANCE extends LogKey - case object RELEASED_LOCKS extends LogKey - case object REMAINING_PARTITIONS extends LogKey - case object REMOTE_ADDRESS extends LogKey - case object REMOTE_BLOCKS_SIZE extends LogKey - case object REMOVE_FROM_MASTER extends LogKey - case object REPORT_DETAILS extends LogKey - case object REQUESTER_SIZE extends LogKey - case object REQUEST_EXECUTORS extends LogKey - case object REQUEST_ID extends LogKey - case object RESOURCE extends LogKey - case object RESOURCE_NAME extends LogKey - case object RESOURCE_PROFILE_ID extends LogKey - case object RESOURCE_PROFILE_IDS extends LogKey - case object RESOURCE_PROFILE_TO_TOTAL_EXECS extends LogKey - case object RESPONSE_BODY_SIZE extends LogKey - case object RESTART_TIME extends LogKey - case object RESULT extends LogKey - case object RESULT_SIZE_BYTES extends LogKey - case object RESULT_SIZE_BYTES_MAX extends LogKey - case object RETRY_INTERVAL extends LogKey - case object RETRY_WAIT_TIME extends LogKey - case object RIGHT_EXPR extends LogKey - case object RIGHT_LOGICAL_PLAN_STATS_SIZE_IN_BYTES extends LogKey - case object RMSE extends LogKey - case object ROCKS_DB_FILE_MAPPING extends LogKey - case object ROCKS_DB_LOG_LEVEL extends LogKey - case object ROCKS_DB_LOG_MESSAGE extends LogKey - case object RPC_ADDRESS extends LogKey - case object RPC_ENDPOINT_REF extends LogKey - case object RPC_MESSAGE_CAPACITY extends LogKey - case object RPC_SSL_ENABLED extends LogKey - case object RULE_EXECUTOR_NAME extends LogKey - case object RULE_NAME extends LogKey - case object RUN_ID extends LogKey - case object RUN_ID_STRING extends LogKey - case object SCALA_VERSION extends LogKey - case object SCALING_DOWN_RATIO extends LogKey - case object SCALING_UP_RATIO extends LogKey - case object SCHEDULER_POOL_NAME extends LogKey - case object SCHEDULING_MODE extends LogKey - case object SCHEMA extends LogKey - case object SCHEMA2 extends LogKey - case object SERVER_NAME extends LogKey - case object SERVICE_NAME extends LogKey - case object SERVLET_CONTEXT_HANDLER_PATH extends LogKey - case object SESSION_HANDLE extends LogKey - case object SESSION_HOLD_INFO extends LogKey - case object SESSION_ID extends LogKey - case object SESSION_KEY extends LogKey - case object SET_CLIENT_INFO_REQUEST extends LogKey - case object SHARD_ID extends LogKey - case object SHORTER_SERVICE_NAME extends LogKey - case object SHORT_USER_NAME extends LogKey - case object SHUFFLE_BLOCK_INFO extends LogKey - case object SHUFFLE_DB_BACKEND_KEY extends LogKey - case object SHUFFLE_DB_BACKEND_NAME extends LogKey - case object SHUFFLE_ID extends LogKey - case object SHUFFLE_IDS extends LogKey - case object SHUFFLE_MERGE_ID extends LogKey - case object SHUFFLE_MERGE_RECOVERY_FILE extends LogKey - case object SHUFFLE_SERVICE_CONF_OVERLAY_URL extends LogKey - case object SHUFFLE_SERVICE_METRICS_NAMESPACE extends LogKey - case object SHUFFLE_SERVICE_NAME extends LogKey - case object SIGMAS_LENGTH extends LogKey - case object SIGNAL extends LogKey - case object SINK extends LogKey - case object SIZE extends LogKey - case object SLEEP_TIME extends LogKey - case object SLIDE_DURATION extends LogKey - case object SMALLEST_CLUSTER_INDEX extends LogKey - case object SNAPSHOT_EVENT extends LogKey - case object SNAPSHOT_EVENT_TIME_DELTA extends LogKey - case object SNAPSHOT_EVENT_VERSION_DELTA extends LogKey - case object SNAPSHOT_VERSION extends LogKey - case object SOCKET_ADDRESS extends LogKey - case object SOURCE extends LogKey - case object SOURCE_PATH extends LogKey - case object SPARK_BRANCH extends LogKey - case object SPARK_BUILD_DATE extends LogKey - case object SPARK_BUILD_USER extends LogKey - case object SPARK_DATA_STREAM extends LogKey - case object SPARK_PLAN_ID extends LogKey - case object SPARK_REPO_URL extends LogKey - case object SPARK_REVISION extends LogKey - case object SPARK_VERSION extends LogKey - case object SPILL_RECORDS_SIZE extends LogKey - case object SPILL_RECORDS_SIZE_THRESHOLD extends LogKey - case object SPILL_TIMES extends LogKey - case object SQL_TEXT extends LogKey - case object SRC_PATH extends LogKey - case object STAGE extends LogKey - case object STAGES extends LogKey - case object STAGE_ATTEMPT extends LogKey - case object STAGE_ATTEMPT_ID extends LogKey - case object STAGE_ID extends LogKey - case object STAGE_NAME extends LogKey - case object STAMP extends LogKey - case object START_INDEX extends LogKey - case object START_TIME extends LogKey - case object STATEMENT_ID extends LogKey - case object STATE_NAME extends LogKey - case object STATE_STORE_COORDINATOR extends LogKey - case object STATE_STORE_ID extends LogKey - case object STATE_STORE_PROVIDER extends LogKey - case object STATE_STORE_PROVIDER_ID extends LogKey - case object STATE_STORE_PROVIDER_IDS extends LogKey - case object STATE_STORE_STATE extends LogKey - case object STATE_STORE_VERSION extends LogKey - case object STATS extends LogKey - case object STATUS extends LogKey - case object STDERR extends LogKey - case object STOP_SITE_SHORT_FORM extends LogKey - case object STORAGE_LEVEL extends LogKey - case object STORAGE_LEVEL_DESERIALIZED extends LogKey - case object STORAGE_LEVEL_REPLICATION extends LogKey - case object STORAGE_MEMORY_SIZE extends LogKey - case object STORE_ID extends LogKey - case object STRATEGY extends LogKey - case object STREAMING_CONTEXT extends LogKey - case object STREAMING_DATA_SOURCE_DESCRIPTION extends LogKey - case object STREAMING_DATA_SOURCE_NAME extends LogKey - case object STREAMING_OFFSETS_END extends LogKey - case object STREAMING_OFFSETS_START extends LogKey - case object STREAMING_QUERY_PROGRESS extends LogKey - case object STREAMING_SOURCE extends LogKey - case object STREAMING_TABLE extends LogKey - case object STREAMING_WRITE extends LogKey - case object STREAM_CHUNK_ID extends LogKey - case object STREAM_ID extends LogKey - case object STREAM_NAME extends LogKey - case object SUBMISSION_ID extends LogKey - case object SUBSAMPLING_RATE extends LogKey - case object SUB_QUERY extends LogKey - case object TABLE_NAME extends LogKey - case object TABLE_TYPE extends LogKey - case object TABLE_TYPES extends LogKey - case object TAG extends LogKey - case object TARGET_NUM_EXECUTOR extends LogKey - case object TARGET_NUM_EXECUTOR_DELTA extends LogKey - case object TARGET_PATH extends LogKey - case object TARGET_SIZE extends LogKey - case object TASK_ATTEMPT_ID extends LogKey - case object TASK_ID extends LogKey - case object TASK_INDEX extends LogKey - case object TASK_LOCALITY extends LogKey - case object TASK_NAME extends LogKey - case object TASK_REQUIREMENTS extends LogKey - case object TASK_RESOURCES extends LogKey - case object TASK_RESOURCE_ASSIGNMENTS extends LogKey - case object TASK_SET_MANAGER extends LogKey - case object TASK_SET_NAME extends LogKey - case object TASK_STATE extends LogKey - case object TEMP_FILE extends LogKey - case object TEMP_OUTPUT_PATH extends LogKey - case object TEMP_PATH extends LogKey - case object TEST_SIZE extends LogKey - case object THREAD extends LogKey - case object THREAD_ID extends LogKey - case object THREAD_NAME extends LogKey - case object THREAD_POOL_KEEPALIVE_TIME extends LogKey - case object THREAD_POOL_SIZE extends LogKey - case object THREAD_POOL_WAIT_QUEUE_SIZE extends LogKey - case object THRESHOLD extends LogKey - case object THRESH_TIME extends LogKey - case object TIME extends LogKey - case object TIMEOUT extends LogKey - case object TIMER extends LogKey - case object TIMESTAMP extends LogKey - case object TIME_UNITS extends LogKey - case object TIP extends LogKey - case object TOKEN extends LogKey - case object TOKEN_KIND extends LogKey - case object TOKEN_REGEX extends LogKey - case object TOKEN_RENEWER extends LogKey - case object TOPIC extends LogKey - case object TOPIC_PARTITION extends LogKey - case object TOPIC_PARTITIONS extends LogKey - case object TOPIC_PARTITION_OFFSET extends LogKey - case object TOPIC_PARTITION_OFFSET_RANGE extends LogKey - case object TOTAL extends LogKey - case object TOTAL_EFFECTIVE_TIME extends LogKey - case object TOTAL_SIZE extends LogKey - case object TOTAL_TIME extends LogKey - case object TOTAL_TIME_READ extends LogKey - case object TO_TIME extends LogKey - case object TRAINING_SIZE extends LogKey - case object TRAIN_VALIDATION_SPLIT_METRIC extends LogKey - case object TRAIN_VALIDATION_SPLIT_METRICS extends LogKey - case object TRANSFER_TYPE extends LogKey - case object TREE_NODE extends LogKey - case object TRIGGER_INTERVAL extends LogKey - case object UI_ACLS extends LogKey - case object UI_FILTER extends LogKey - case object UI_FILTER_PARAMS extends LogKey - case object UI_PROXY_BASE extends LogKey - case object UNKNOWN_PARAM extends LogKey - case object UNSUPPORTED_EXPR extends LogKey - case object UNSUPPORTED_HINT_REASON extends LogKey - case object UNTIL_OFFSET extends LogKey - case object UPPER_BOUND extends LogKey - case object URI extends LogKey - case object URIS extends LogKey - case object URL extends LogKey - case object URL2 extends LogKey - case object URLS extends LogKey - case object USER_ID extends LogKey - case object USER_NAME extends LogKey - case object UUID extends LogKey - case object VALUE extends LogKey - case object VERSIONS_TO_DELETE extends LogKey - case object VERSION_NUM extends LogKey - case object VIEW_ACLS extends LogKey - case object VIEW_ACLS_GROUPS extends LogKey - case object VIRTUAL_CORES extends LogKey - case object VOCAB_SIZE extends LogKey - case object WAIT_RESULT_TIME extends LogKey - case object WAIT_SEND_TIME extends LogKey - case object WATERMARK_CONSTRAINT extends LogKey - case object WEB_URL extends LogKey - case object WEIGHT extends LogKey - case object WORKER extends LogKey - case object WORKER_HOST extends LogKey - case object WORKER_ID extends LogKey - case object WORKER_PORT extends LogKey - case object WORKER_URL extends LogKey - case object WRITE_AHEAD_LOG_INFO extends LogKey - case object WRITE_AHEAD_LOG_RECORD_HANDLE extends LogKey - case object WRITE_JOB_UUID extends LogKey - case object XML_SCHEDULING_MODE extends LogKey - case object XSD_PATH extends LogKey - case object YARN_RESOURCE extends LogKey - case object YOUNG_GENERATION_GC extends LogKey - case object ZERO_TIME extends LogKey -} diff --git a/common/utils/src/main/scala/org/apache/spark/internal/Logging.scala b/common/utils/src/main/scala/org/apache/spark/internal/Logging.scala index cc5d0281829d0..810bdabebb38a 100644 --- a/common/utils/src/main/scala/org/apache/spark/internal/Logging.scala +++ b/common/utils/src/main/scala/org/apache/spark/internal/Logging.scala @@ -17,6 +17,8 @@ package org.apache.spark.internal +import java.util.Locale + import scala.concurrent.duration._ import scala.jdk.CollectionConverters._ @@ -29,7 +31,6 @@ import org.slf4j.{Logger, LoggerFactory} import org.slf4j.event.{Level => Slf4jLevel} import org.apache.spark.internal.Logging.SparkShellLoggingFilter -import org.apache.spark.internal.LogKeys import org.apache.spark.util.SparkClassUtils /** @@ -61,29 +62,16 @@ import org.apache.spark.util.SparkClassUtils *

* * If you want to output logs in `scala code` through the structured log framework, - * you can define `custom LogKey` and use it in `scala` code as follows: + * you can define `custom LogKey` in `java` and use it in `scala` code as follows: *

* - * // To add a `custom LogKey`, implement `LogKey` - * case object CUSTOM_LOG_KEY extends LogKey - * import org.apache.spark.internal.MDC; + * // Add a `CustomLogKeys`, implement `LogKey` + * public enum CustomLogKeys implements LogKey { + * CUSTOM_LOG_KEY + * } * logInfo(log"${MDC(CUSTOM_LOG_KEY, "key")}") */ -/** - * Mapped Diagnostic Context (MDC) that will be used in log messages. - * The values of the MDC will be inline in the log message, while the key-value pairs will be - * part of the ThreadContext. - */ -case class MDC(key: LogKey, value: Any) { - require(!value.isInstanceOf[MessageWithContext], - "the class of value cannot be MessageWithContext") -} - -object MDC { - def of(key: LogKey, value: Any): MDC = MDC(key, value) -} - /** * Wrapper class for log messages that include a logging context. * This is used as the return type of the string interpolator `LogStringContext`. @@ -155,7 +143,7 @@ trait Logging { val value = if (mdc.value != null) mdc.value.toString else null sb.append(value) if (Logging.isStructuredLoggingEnabled) { - context.put(mdc.key.name, value) + context.put(mdc.key.name.toLowerCase(Locale.ROOT), value) } if (processedParts.hasNext) { @@ -182,6 +170,12 @@ trait Logging { } } + protected def MDC(key: LogKey, value: Any): MDC = { + require(!value.isInstanceOf[MessageWithContext], + "the class of value cannot be MessageWithContext") + new MDC(key, value) + } + // Log methods that take only a String protected def logInfo(msg: => String): Unit = { if (log.isInfoEnabled) log.info(msg) @@ -411,7 +405,6 @@ private[spark] object Logging { @volatile private var initialized = false @volatile private var defaultRootLevel: Level = null @volatile private var defaultSparkLog4jConfig = false - @volatile private var structuredLoggingEnabled = false @volatile private[spark] var sparkShellThresholdLevel: Level = null @volatile private[spark] var setLogLevelPrinted: Boolean = false @@ -484,21 +477,21 @@ private[spark] object Logging { * Enable Structured logging framework. */ private[spark] def enableStructuredLogging(): Unit = { - structuredLoggingEnabled = true + SparkLoggerFactory.enableStructuredLogging() } /** * Disable Structured logging framework. */ private[spark] def disableStructuredLogging(): Unit = { - structuredLoggingEnabled = false + SparkLoggerFactory.disableStructuredLogging() } /** * Return true if Structured logging framework is enabled. */ private[spark] def isStructuredLoggingEnabled: Boolean = { - structuredLoggingEnabled + SparkLoggerFactory.isStructuredLoggingEnabled } private[spark] class SparkShellLoggingFilter extends AbstractFilter { diff --git a/common/utils/src/main/scala/org/apache/spark/util/MavenUtils.scala b/common/utils/src/main/scala/org/apache/spark/util/MavenUtils.scala index f101cd81225d5..5e923ad35a3ab 100644 --- a/common/utils/src/main/scala/org/apache/spark/util/MavenUtils.scala +++ b/common/utils/src/main/scala/org/apache/spark/util/MavenUtils.scala @@ -35,7 +35,7 @@ import org.apache.ivy.plugins.repository.file.FileRepository import org.apache.ivy.plugins.resolver.{ChainResolver, FileSystemResolver, IBiblioResolver} import org.apache.spark.SparkException -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.util.ArrayImplicits._ /** Provides utility functions to be used inside SparkSubmit. */ diff --git a/common/utils/src/main/scala/org/apache/spark/util/SparkErrorUtils.scala b/common/utils/src/main/scala/org/apache/spark/util/SparkErrorUtils.scala index c2b56fe85b7fa..4c8721050796e 100644 --- a/common/utils/src/main/scala/org/apache/spark/util/SparkErrorUtils.scala +++ b/common/utils/src/main/scala/org/apache/spark/util/SparkErrorUtils.scala @@ -23,7 +23,7 @@ import scala.annotation.tailrec import scala.collection.mutable import scala.util.control.NonFatal -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} private[spark] trait SparkErrorUtils extends Logging { /** diff --git a/common/utils/src/main/scala/org/apache/spark/util/SparkFileUtils.scala b/common/utils/src/main/scala/org/apache/spark/util/SparkFileUtils.scala index 9e63d74fe88c0..42df312bc9a06 100644 --- a/common/utils/src/main/scala/org/apache/spark/util/SparkFileUtils.scala +++ b/common/utils/src/main/scala/org/apache/spark/util/SparkFileUtils.scala @@ -21,7 +21,7 @@ import java.net.{URI, URISyntaxException} import java.nio.file.{Files, Path, StandardCopyOption} import java.nio.file.attribute.FileTime -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.network.util.JavaUtils private[spark] trait SparkFileUtils extends Logging { diff --git a/common/utils/src/test/java/org/apache/spark/util/CustomLogKeys.java b/common/utils/src/test/java/org/apache/spark/util/CustomLogKeys.java new file mode 100644 index 0000000000000..cadacba7c5175 --- /dev/null +++ b/common/utils/src/test/java/org/apache/spark/util/CustomLogKeys.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util; + +import org.apache.spark.internal.LogKey; + +public enum CustomLogKeys implements LogKey { + CUSTOM_LOG_KEY +} diff --git a/common/utils/src/test/java/org/apache/spark/util/PatternSparkLoggerSuite.java b/common/utils/src/test/java/org/apache/spark/util/PatternSparkLoggerSuite.java index 6bfe595def1d4..7f8f3f93a8d46 100644 --- a/common/utils/src/test/java/org/apache/spark/util/PatternSparkLoggerSuite.java +++ b/common/utils/src/test/java/org/apache/spark/util/PatternSparkLoggerSuite.java @@ -90,12 +90,7 @@ String expectedPatternForMsgWithMDCValueIsNull(Level level) { } @Override - String expectedPatternForScalaCustomLogKey(Level level) { - return toRegexPattern(level, ".* : Scala custom log message.\n"); - } - - @Override - String expectedPatternForJavaCustomLogKey(Level level) { - return toRegexPattern(level, ".* : Java custom log message.\n"); + String expectedPatternForCustomLogKey(Level level) { + return toRegexPattern(level, ".* : Custom log message.\n"); } } diff --git a/common/utils/src/test/java/org/apache/spark/util/SparkLoggerSuiteBase.java b/common/utils/src/test/java/org/apache/spark/util/SparkLoggerSuiteBase.java index ad28d29bdba34..d86fe12c89243 100644 --- a/common/utils/src/test/java/org/apache/spark/util/SparkLoggerSuiteBase.java +++ b/common/utils/src/test/java/org/apache/spark/util/SparkLoggerSuiteBase.java @@ -25,7 +25,6 @@ import org.apache.logging.log4j.Level; import org.junit.jupiter.api.Test; -import org.apache.spark.internal.LogKey; import org.apache.spark.internal.LogKeys; import org.apache.spark.internal.MDC; import org.apache.spark.internal.SparkLogger; @@ -72,23 +71,20 @@ private void checkLogOutput(Level level, Runnable func, ExpectedResult result) { private final String basicMsgWithEscapeChar = "This is a log message\nThis is a new line \t other msg"; - private final MDC executorIDMDC = MDC.of(LogKeys.EXECUTOR_ID$.MODULE$, "1"); + private final MDC executorIDMDC = MDC.of(LogKeys.EXECUTOR_ID, "1"); private final String msgWithMDC = "Lost executor {}."; private final MDC[] mdcs = new MDC[] { - MDC.of(LogKeys.EXECUTOR_ID$.MODULE$, "1"), - MDC.of(LogKeys.REASON$.MODULE$, "the shuffle data is too large")}; + MDC.of(LogKeys.EXECUTOR_ID, "1"), + MDC.of(LogKeys.REASON, "the shuffle data is too large")}; private final String msgWithMDCs = "Lost executor {}, reason: {}"; private final MDC[] emptyMDCs = new MDC[0]; - private final MDC executorIDMDCValueIsNull = MDC.of(LogKeys.EXECUTOR_ID$.MODULE$, null); + private final MDC executorIDMDCValueIsNull = MDC.of(LogKeys.EXECUTOR_ID, null); - private final MDC scalaCustomLogMDC = - MDC.of(CustomLogKeys.CUSTOM_LOG_KEY$.MODULE$, "Scala custom log message."); - - private final MDC javaCustomLogMDC = - MDC.of(JavaCustomLogKeys.CUSTOM_LOG_KEY, "Java custom log message."); + private final MDC customLogMDC = + MDC.of(CustomLogKeys.CUSTOM_LOG_KEY, "Custom log message."); // test for basic message (without any mdc) abstract String expectedPatternForBasicMsg(Level level); @@ -117,10 +113,7 @@ String expectedPatternForMsgWithEmptyMDCsAndException(Level level) { abstract String expectedPatternForMsgWithMDCValueIsNull(Level level); // test for scala custom LogKey - abstract String expectedPatternForScalaCustomLogKey(Level level); - - // test for java custom LogKey - abstract String expectedPatternForJavaCustomLogKey(Level level); + abstract String expectedPatternForCustomLogKey(Level level); @Test public void testBasicMsg() { @@ -240,34 +233,14 @@ public void testLoggerWithMDCValueIsNull() { } @Test - public void testLoggerWithScalaCustomLogKey() { - Runnable errorFn = () -> logger().error("{}", scalaCustomLogMDC); - Runnable warnFn = () -> logger().warn("{}", scalaCustomLogMDC); - Runnable infoFn = () -> logger().info("{}", scalaCustomLogMDC); - List.of( - Pair.of(Level.ERROR, errorFn), - Pair.of(Level.WARN, warnFn), - Pair.of(Level.INFO, infoFn)).forEach(pair -> - checkLogOutput(pair.getLeft(), pair.getRight(), this::expectedPatternForScalaCustomLogKey)); - } - - @Test - public void testLoggerWithJavaCustomLogKey() { - Runnable errorFn = () -> logger().error("{}", javaCustomLogMDC); - Runnable warnFn = () -> logger().warn("{}", javaCustomLogMDC); - Runnable infoFn = () -> logger().info("{}", javaCustomLogMDC); + public void testLoggerWithCustomLogKey() { + Runnable errorFn = () -> logger().error("{}", customLogMDC); + Runnable warnFn = () -> logger().warn("{}", customLogMDC); + Runnable infoFn = () -> logger().info("{}", customLogMDC); List.of( Pair.of(Level.ERROR, errorFn), Pair.of(Level.WARN, warnFn), Pair.of(Level.INFO, infoFn)).forEach(pair -> - checkLogOutput(pair.getLeft(), pair.getRight(), this::expectedPatternForJavaCustomLogKey)); + checkLogOutput(pair.getLeft(), pair.getRight(), this::expectedPatternForCustomLogKey)); } } - -class JavaCustomLogKeys { - // Custom `LogKey` must be `implements LogKey` - public static class CUSTOM_LOG_KEY implements LogKey { } - - // Singleton - public static final CUSTOM_LOG_KEY CUSTOM_LOG_KEY = new CUSTOM_LOG_KEY(); -} diff --git a/common/utils/src/test/java/org/apache/spark/util/StructuredSparkLoggerSuite.java b/common/utils/src/test/java/org/apache/spark/util/StructuredSparkLoggerSuite.java index 1fab167adfeb0..cc2d156cc9348 100644 --- a/common/utils/src/test/java/org/apache/spark/util/StructuredSparkLoggerSuite.java +++ b/common/utils/src/test/java/org/apache/spark/util/StructuredSparkLoggerSuite.java @@ -176,28 +176,14 @@ String expectedPatternForMsgWithMDCValueIsNull(Level level) { } @Override - String expectedPatternForScalaCustomLogKey(Level level) { + String expectedPatternForCustomLogKey(Level level) { return compactAndToRegexPattern(level, """ { "ts": "", "level": "", - "msg": "Scala custom log message.", + "msg": "Custom log message.", "context": { - "custom_log_key": "Scala custom log message." - }, - "logger": "" - }"""); - } - - @Override - String expectedPatternForJavaCustomLogKey(Level level) { - return compactAndToRegexPattern(level, """ - { - "ts": "", - "level": "", - "msg": "Java custom log message.", - "context": { - "custom_log_key": "Java custom log message." + "custom_log_key": "Custom log message." }, "logger": "" }"""); diff --git a/common/utils/src/test/scala/org/apache/spark/util/LogKeySuite.scala b/common/utils/src/test/scala/org/apache/spark/util/LogKeysSuite.scala similarity index 63% rename from common/utils/src/test/scala/org/apache/spark/util/LogKeySuite.scala rename to common/utils/src/test/scala/org/apache/spark/util/LogKeysSuite.scala index 742d4066ffab2..25a908a8b96e2 100644 --- a/common/utils/src/test/scala/org/apache/spark/util/LogKeySuite.scala +++ b/common/utils/src/test/scala/org/apache/spark/util/LogKeysSuite.scala @@ -17,25 +17,25 @@ package org.apache.spark.util +import java.nio.charset.StandardCharsets import java.nio.file.{Files, Path} -import java.util.{ArrayList => JList} import scala.jdk.CollectionConverters._ -import scala.reflect.runtime.universe._ import org.scalatest.funsuite.AnyFunSuite // scalastyle:ignore funsuite import org.apache.spark.internal.{Logging, LogKeys} +import org.apache.spark.util.ArrayImplicits._ // scalastyle:off line.size.limit /** - * To re-generate the file `LogKey.scala`, run: + * To re-generate the file `LogKeys.java`, run: * {{{ - * SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "common-utils/testOnly org.apache.spark.util.LogKeySuite" + * SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "common-utils/testOnly org.apache.spark.util.LogKeysSuite" * }}} */ // scalastyle:on line.size.limit -class LogKeySuite +class LogKeysSuite extends AnyFunSuite // scalastyle:ignore funsuite with Logging { @@ -52,36 +52,33 @@ class LogKeySuite private val regenerateGoldenFiles: Boolean = System.getenv("SPARK_GENERATE_GOLDEN_FILES") == "1" - private val logKeyFilePath = getWorkspaceFilePath("common", "utils", "src", "main", "scala", - "org", "apache", "spark", "internal", "LogKey.scala") + private val logKeyFilePath = getWorkspaceFilePath("common", "utils", "src", "main", "java", + "org", "apache", "spark", "internal", "LogKeys.java") - // regenerate the file `LogKey.scala` with its members sorted alphabetically + // regenerate the file `LogKeys.java` with its members sorted alphabetically private def regenerateLogKeyFile( originalKeys: Seq[String], sortedKeys: Seq[String]): Unit = { if (originalKeys != sortedKeys) { - logInfo(s"Regenerating the file $logKeyFilePath") - val originalContents = Files.readAllLines(logKeyFilePath) - val sortedContents = new JList[String]() - var firstMatch = false - originalContents.asScala.foreach { line => - if (line.trim.startsWith("case object ") && line.trim.endsWith(" extends LogKey")) { - if (!firstMatch) { - sortedKeys.foreach { key => - sortedContents.add(s" case object $key extends LogKey") - } - firstMatch = true - } - } else { - sortedContents.add(line) - } - } - Files.delete(logKeyFilePath) - Files.write(logKeyFilePath, sortedContents) + val logKeyFile = logKeyFilePath.toFile + logInfo(s"Regenerating the file $logKeyFile") + val originalContents = Files.readAllLines(logKeyFile.toPath, StandardCharsets.UTF_8).asScala + + val beforeFirstIndex = originalContents + .indexWhere(_.contains("public enum LogKeys implements LogKey")) + + val content = + s"""${originalContents.take(beforeFirstIndex + 1).mkString("\n")} + |${sortedKeys.map { key => s" $key" }.mkString(",\n")} + |} + |""".stripMargin + + Files.delete(logKeyFile.toPath) + Files.writeString(logKeyFile.toPath, content, StandardCharsets.UTF_8) } } test("The members of LogKeys are correctly sorted") { - val originalKeys = getAllLogKeys.reverse + val originalKeys = LogKeys.values.map(_.name).toImmutableArraySeq val sortedKeys = originalKeys.sorted if (regenerateGoldenFiles) { regenerateLogKeyFile(originalKeys, sortedKeys) @@ -90,11 +87,4 @@ class LogKeySuite "The members of LogKeys must be sorted alphabetically") } } - - private def getAllLogKeys: Seq[String] = { - val logKeysType = typeOf[LogKeys.type] - val classSymbol = logKeysType.typeSymbol.asClass - val members = classSymbol.typeSignature.members - members.filter(m => m.isTerm && !m.isMethod).map(_.name.toString).toSeq - } } diff --git a/common/utils/src/test/scala/org/apache/spark/util/MDCSuite.scala b/common/utils/src/test/scala/org/apache/spark/util/MDCSuite.scala index 9615eb2263636..f086b6d18dafc 100644 --- a/common/utils/src/test/scala/org/apache/spark/util/MDCSuite.scala +++ b/common/utils/src/test/scala/org/apache/spark/util/MDCSuite.scala @@ -22,7 +22,7 @@ import scala.jdk.CollectionConverters._ import org.scalatest.BeforeAndAfterAll import org.scalatest.funsuite.AnyFunSuite // scalastyle:ignore funsuite -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{EXIT_CODE, OFFSET, RANGE} class MDCSuite diff --git a/common/utils/src/test/scala/org/apache/spark/util/StructuredLoggingSuite.scala b/common/utils/src/test/scala/org/apache/spark/util/StructuredLoggingSuite.scala index 0026b696f0695..20f6d2347eabe 100644 --- a/common/utils/src/test/scala/org/apache/spark/util/StructuredLoggingSuite.scala +++ b/common/utils/src/test/scala/org/apache/spark/util/StructuredLoggingSuite.scala @@ -26,7 +26,7 @@ import org.apache.logging.log4j.Level import org.scalatest.BeforeAndAfterAll import org.scalatest.funsuite.AnyFunSuite // scalastyle:ignore funsuite -import org.apache.spark.internal.{LogEntry, Logging, LogKey, LogKeys, MDC, MessageWithContext} +import org.apache.spark.internal.{LogEntry, Logging, LogKeys, MessageWithContext} trait LoggingSuiteBase extends AnyFunSuite // scalastyle:ignore funsuite @@ -491,8 +491,3 @@ class StructuredLoggingSuite extends LoggingSuiteBase { assert(mdcPattern.r.findFirstIn(logOutputWithoutMDCSet).isEmpty) } } - -object CustomLogKeys { - // Custom `LogKey` must be `extends LogKey` - case object CUSTOM_LOG_KEY extends LogKey -} diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala index 1df01bd3bfb62..0062b03575930 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala @@ -37,7 +37,6 @@ import org.scalatest.concurrent.{Eventually, PatienceConfiguration} import org.scalatest.time.SpanSugar._ import org.apache.spark.internal.LogKeys.{CLASS_NAME, CONTAINER, STATUS} -import org.apache.spark.internal.MDC import org.apache.spark.sql.QueryTest import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.{DockerUtils, Utils} diff --git a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchPartitionReader.scala b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchPartitionReader.scala index cb1c7055483b3..9a7c9f2c6c6a7 100644 --- a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchPartitionReader.scala +++ b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchPartitionReader.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} import org.apache.spark.TaskContext -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeRow diff --git a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala index 10bdbb1d9d447..041fe074f7e40 100644 --- a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala +++ b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala @@ -24,7 +24,7 @@ import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, Offset import org.apache.kafka.common.TopicPartition import org.apache.spark.TaskContext -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{ERROR, OFFSETS, TIP} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeRow diff --git a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala index c79da13017b97..19a5d7cb739f5 100644 --- a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala +++ b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala @@ -25,7 +25,7 @@ import scala.jdk.CollectionConverters._ import org.apache.kafka.common.TopicPartition import org.apache.spark.SparkEnv -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{ERROR, OFFSETS, TIP} import org.apache.spark.internal.config.Network.NETWORK_TIMEOUT import org.apache.spark.sql.SparkSession diff --git a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderAdmin.scala b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderAdmin.scala index 86cec6fc041cd..7420c2c1055b2 100644 --- a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderAdmin.scala +++ b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderAdmin.scala @@ -30,7 +30,7 @@ import org.apache.kafka.common.{IsolationLevel, TopicPartition} import org.apache.kafka.common.requests.OffsetFetchResponse import org.apache.spark.SparkEnv -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{NUM_RETRY, OFFSETS, TOPIC_PARTITION_OFFSET} import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap diff --git a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderConsumer.scala b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderConsumer.scala index fb06797d1fe34..b7ac9a171c575 100644 --- a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderConsumer.scala +++ b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderConsumer.scala @@ -27,7 +27,7 @@ import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, OffsetAndTim import org.apache.kafka.common.TopicPartition import org.apache.spark.SparkEnv -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{NUM_RETRY, OFFSETS, TOPIC_PARTITION_OFFSET} import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap diff --git a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala index ff884310f660c..7e98446567640 100644 --- a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala +++ b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.kafka010 -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.TOPIC_PARTITIONS import org.apache.spark.internal.config.Network.NETWORK_TIMEOUT import org.apache.spark.rdd.RDD diff --git a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSink.scala b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSink.scala index fb473e71d5a75..d3ea5332f4e44 100644 --- a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSink.scala +++ b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSink.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.BATCH_ID import org.apache.spark.sql.DataFrame import org.apache.spark.sql.execution.streaming.Sink diff --git a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index 1b52046b14833..b2679e27827fe 100644 --- a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -22,7 +22,7 @@ import java.{util => ju} import org.apache.kafka.common.TopicPartition import org.apache.spark.SparkContext -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{ERROR, FROM_OFFSET, OFFSETS, TIP, TOPIC_PARTITIONS, UNTIL_OFFSET} import org.apache.spark.internal.config.Network.NETWORK_TIMEOUT import org.apache.spark.scheduler.ExecutorCacheTaskLocation diff --git a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala index 82ad75e028afe..ff2f16d26b932 100644 --- a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala +++ b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -27,7 +27,7 @@ import org.apache.kafka.clients.producer.ProducerConfig import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.serialization.{ByteArrayDeserializer, ByteArraySerializer} -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.kafka010.KafkaConfigUpdater import org.apache.spark.sql.{AnalysisException, DataFrame, SaveMode, SQLContext} import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap diff --git a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala index 0f962a29588a0..9bae686a65bf8 100644 --- a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala +++ b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala @@ -23,7 +23,6 @@ import org.apache.kafka.clients.consumer.ConsumerRecord import org.apache.spark.{Partition, SparkContext, SparkException, TaskContext} import org.apache.spark.internal.LogKeys.{FROM_OFFSET, PARTITION_ID, TOPIC} -import org.apache.spark.internal.MDC import org.apache.spark.rdd.RDD import org.apache.spark.sql.kafka010.consumer.KafkaDataConsumer import org.apache.spark.storage.StorageLevel diff --git a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/FetchedDataPool.scala b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/FetchedDataPool.scala index 9f68cb6fd0882..2ea1104177a54 100644 --- a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/FetchedDataPool.scala +++ b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/FetchedDataPool.scala @@ -26,7 +26,7 @@ import scala.collection.mutable import org.apache.kafka.clients.consumer.ConsumerRecord import org.apache.spark.SparkConf -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{DATA, KEY} import org.apache.spark.sql.kafka010.{FETCHED_DATA_CACHE_EVICTOR_THREAD_RUN_INTERVAL, FETCHED_DATA_CACHE_TIMEOUT} import org.apache.spark.sql.kafka010.consumer.KafkaDataConsumer.{AvailableOffsetRange, CacheKey, UNKNOWN_OFFSET} diff --git a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/KafkaDataConsumer.scala b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/KafkaDataConsumer.scala index ceb9d96660ae3..2d1125294df27 100644 --- a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/KafkaDataConsumer.scala +++ b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/KafkaDataConsumer.scala @@ -30,7 +30,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.deploy.security.HadoopDelegationTokenManager -import org.apache.spark.internal.{Logging, MDC, MessageWithContext} +import org.apache.spark.internal.{Logging, MessageWithContext} import org.apache.spark.internal.LogKeys._ import org.apache.spark.kafka010.{KafkaConfigUpdater, KafkaTokenUtil} import org.apache.spark.sql.catalyst.util.DateTimeConstants.NANOS_PER_MILLIS diff --git a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/producer/CachedKafkaProducer.scala b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/producer/CachedKafkaProducer.scala index c3457cf8982d9..9b40bcf8ce779 100644 --- a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/producer/CachedKafkaProducer.scala +++ b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/producer/CachedKafkaProducer.scala @@ -23,7 +23,7 @@ import scala.util.control.NonFatal import org.apache.kafka.clients.producer.KafkaProducer -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.PRODUCER_ID private[kafka010] class CachedKafkaProducer( diff --git a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/producer/InternalKafkaProducerPool.scala b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/producer/InternalKafkaProducerPool.scala index 79e0a91dd8968..ca9a0059ba1e4 100644 --- a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/producer/InternalKafkaProducerPool.scala +++ b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/producer/InternalKafkaProducerPool.scala @@ -27,7 +27,7 @@ import scala.jdk.CollectionConverters._ import org.apache.kafka.clients.producer.KafkaProducer import org.apache.spark.{SparkConf, SparkEnv} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.PRODUCER_ID import org.apache.spark.kafka010.{KafkaConfigUpdater, KafkaRedactionUtil} import org.apache.spark.sql.kafka010.{PRODUCER_CACHE_EVICTOR_THREAD_RUN_INTERVAL, PRODUCER_CACHE_TIMEOUT} diff --git a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala index 60de3705636ec..ee4ab5724c652 100644 --- a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala +++ b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala @@ -53,7 +53,7 @@ import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys import org.apache.spark.kafka010.KafkaTokenUtil import org.apache.spark.util.{SecurityUtils, ShutdownHookManager, Utils} diff --git a/connector/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaDelegationTokenProvider.scala b/connector/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaDelegationTokenProvider.scala index 8acdd01a15bab..bce318811dc27 100644 --- a/connector/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaDelegationTokenProvider.scala +++ b/connector/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaDelegationTokenProvider.scala @@ -24,7 +24,7 @@ import org.apache.hadoop.security.Credentials import org.apache.kafka.common.security.auth.SecurityProtocol.{SASL_PLAINTEXT, SASL_SSL, SSL} import org.apache.spark.SparkConf -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CLUSTER_ID, SERVICE_NAME} import org.apache.spark.security.HadoopDelegationTokenProvider diff --git a/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala b/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala index 2320f1908da5a..24c3dd0cddf17 100644 --- a/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala +++ b/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala @@ -25,7 +25,7 @@ import scala.jdk.CollectionConverters._ import org.apache.kafka.clients.consumer._ import org.apache.kafka.common.TopicPartition -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.CONFIG import org.apache.spark.kafka010.KafkaConfigUpdater diff --git a/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala b/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala index f7bea064d2d6c..beaac35720445 100644 --- a/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala +++ b/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala @@ -27,7 +27,7 @@ import scala.jdk.CollectionConverters._ import org.apache.kafka.clients.consumer._ import org.apache.kafka.common.TopicPartition -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{OFFSET, TIME, TOPIC_PARTITION, TOPIC_PARTITION_OFFSET_RANGE} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{StreamingContext, Time} diff --git a/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala b/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala index 75b046430ef50..03701dbda0947 100644 --- a/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala +++ b/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala @@ -26,7 +26,7 @@ import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaC import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.spark.TaskContext -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.kafka010.KafkaConfigUpdater diff --git a/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala b/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala index 2637034766574..af97d5464e004 100644 --- a/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala +++ b/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala @@ -23,7 +23,7 @@ import org.apache.kafka.clients.consumer.{ ConsumerConfig, ConsumerRecord } import org.apache.kafka.common.TopicPartition import org.apache.spark.{Partition, SparkContext, TaskContext} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{FROM_OFFSET, PARTITION_ID, TOPIC, UNTIL_OFFSET} import org.apache.spark.internal.config.Network._ import org.apache.spark.partial.{BoundedDouble, PartialResult} diff --git a/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala b/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala index d15e5e25f561d..54cb9ff39d993 100644 --- a/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala +++ b/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala @@ -24,7 +24,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.spark.SparkContext import org.apache.spark.api.java.{ JavaRDD, JavaSparkContext } -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CONFIG, GROUP_ID} import org.apache.spark.rdd.RDD import org.apache.spark.streaming.StreamingContext diff --git a/connector/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala b/connector/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala index b391203b4b968..ac3622f93321a 100644 --- a/connector/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala +++ b/connector/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala @@ -29,7 +29,7 @@ import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord import com.amazonaws.services.kinesis.model._ import org.apache.spark._ -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{ERROR, NUM_RETRY} import org.apache.spark.rdd.{BlockRDD, BlockRDDPartition} import org.apache.spark.storage.BlockId diff --git a/connector/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointer.scala b/connector/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointer.scala index c52eeca1e48a1..b259a5337f37e 100644 --- a/connector/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointer.scala +++ b/connector/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointer.scala @@ -22,7 +22,7 @@ import scala.util.control.NonFatal import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorCheckpointer -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{SHARD_ID, WORKER_URL} import org.apache.spark.streaming.Duration import org.apache.spark.streaming.util.RecurringTimer diff --git a/connector/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala b/connector/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala index 953817e625e48..ab91431035fef 100644 --- a/connector/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala +++ b/connector/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala @@ -28,7 +28,7 @@ import com.amazonaws.services.kinesis.clientlibrary.lib.worker.{KinesisClientLib import com.amazonaws.services.kinesis.metrics.interfaces.MetricsLevel import com.amazonaws.services.kinesis.model.Record -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.WORKER_URL import org.apache.spark.storage.{StorageLevel, StreamBlockId} import org.apache.spark.streaming.Duration diff --git a/connector/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala b/connector/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala index cd740f971e484..8304ddda96dfa 100644 --- a/connector/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala +++ b/connector/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala @@ -26,7 +26,7 @@ import com.amazonaws.services.kinesis.clientlibrary.interfaces.{IRecordProcessor import com.amazonaws.services.kinesis.clientlibrary.lib.worker.ShutdownReason import com.amazonaws.services.kinesis.model.Record -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{KINESIS_REASON, RETRY_INTERVAL, SHARD_ID, WORKER_URL} /** diff --git a/connector/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala b/connector/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala index 652822c5fdc97..3f06d476f08dc 100644 --- a/connector/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala +++ b/connector/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala @@ -33,7 +33,7 @@ import com.amazonaws.services.dynamodbv2.document.DynamoDB import com.amazonaws.services.kinesis.{AmazonKinesis, AmazonKinesisClient} import com.amazonaws.services.kinesis.model._ -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{STREAM_NAME, TABLE_NAME} /** diff --git a/connector/profiler/src/main/scala/org/apache/spark/profiler/ProfilerPlugin.scala b/connector/profiler/src/main/scala/org/apache/spark/profiler/ProfilerPlugin.scala index db3de90099f7c..cb341aef3a0d8 100644 --- a/connector/profiler/src/main/scala/org/apache/spark/profiler/ProfilerPlugin.scala +++ b/connector/profiler/src/main/scala/org/apache/spark/profiler/ProfilerPlugin.scala @@ -23,7 +23,7 @@ import scala.util.Random import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.EXECUTOR_ID /** diff --git a/connector/profiler/src/main/scala/org/apache/spark/profiler/SparkAsyncProfiler.scala b/connector/profiler/src/main/scala/org/apache/spark/profiler/SparkAsyncProfiler.scala index d00d25d8af1ae..02d39a2c435b3 100644 --- a/connector/profiler/src/main/scala/org/apache/spark/profiler/SparkAsyncProfiler.scala +++ b/connector/profiler/src/main/scala/org/apache/spark/profiler/SparkAsyncProfiler.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.fs.permission.FsPermission import org.apache.spark.SparkConf import org.apache.spark.SparkContext.DRIVER_IDENTIFIER import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.PATH import org.apache.spark.util.{ThreadUtils, Utils} diff --git a/connector/spark-ganglia-lgpl/src/main/java/com/codahale/metrics/ganglia/GangliaReporter.java b/connector/spark-ganglia-lgpl/src/main/java/com/codahale/metrics/ganglia/GangliaReporter.java index 48c61e80d6655..3a38a52f7ab5d 100644 --- a/connector/spark-ganglia-lgpl/src/main/java/com/codahale/metrics/ganglia/GangliaReporter.java +++ b/connector/spark-ganglia-lgpl/src/main/java/com/codahale/metrics/ganglia/GangliaReporter.java @@ -295,7 +295,7 @@ private void reportTimer(String name, Timer timer) { reportMetered(sanitizedName, timer, group, "calls"); } catch (GangliaException e) { LOGGER.warn("Unable to report timer {}", e, - MDC.of(LogKeys.METRIC_NAME$.MODULE$, sanitizedName)); + MDC.of(LogKeys.METRIC_NAME, sanitizedName)); } } @@ -306,7 +306,7 @@ private void reportMeter(String name, Meter meter) { reportMetered(sanitizedName, meter, group, "events"); } catch (GangliaException e) { LOGGER.warn("Unable to report meter {}", e, - MDC.of(LogKeys.METRIC_NAME$.MODULE$, name)); + MDC.of(LogKeys.METRIC_NAME, name)); } } @@ -338,7 +338,7 @@ private void reportHistogram(String name, Histogram histogram) { announceIfEnabled(P999, sanitizedName, group, snapshot.get999thPercentile(), ""); } catch (GangliaException e) { LOGGER.warn("Unable to report histogram {}", e, - MDC.of(LogKeys.METRIC_NAME$.MODULE$, sanitizedName)); + MDC.of(LogKeys.METRIC_NAME, sanitizedName)); } } @@ -349,7 +349,7 @@ private void reportCounter(String name, Counter counter) { announce(prefix(sanitizedName, COUNT.getCode()), group, Long.toString(counter.getCount()), GMetricType.DOUBLE, ""); } catch (GangliaException e) { LOGGER.warn("Unable to report counter {}", e, - MDC.of(LogKeys.METRIC_NAME$.MODULE$, name)); + MDC.of(LogKeys.METRIC_NAME, name)); } } @@ -363,7 +363,7 @@ private void reportGauge(String name, Gauge gauge) { announce(name(prefix, sanitizedName), group, value, type, ""); } catch (GangliaException e) { LOGGER.warn("Unable to report gauge {}", e, - MDC.of(LogKeys.METRIC_NAME$.MODULE$, name)); + MDC.of(LogKeys.METRIC_NAME, name)); } } diff --git a/core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java b/core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java index 7dd87df713e6e..96e50955ae2c3 100644 --- a/core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java +++ b/core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java @@ -213,7 +213,7 @@ private void closeUnderlyingInputStreamIfNecessary() { try { underlyingInputStream.close(); } catch (IOException e) { - logger.warn("{}", e, MDC.of(LogKeys.ERROR$.MODULE$, e.getMessage())); + logger.warn("{}", e, MDC.of(LogKeys.ERROR, e.getMessage())); } } } diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java index e98554db22524..ab9e470e0c2c0 100644 --- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java @@ -279,11 +279,11 @@ private long trySpillAndAcquire( } catch (ClosedByInterruptException | InterruptedIOException e) { // This called by user to kill a task (e.g: speculative task). logger.error("Error while calling spill() on {}", e, - MDC.of(LogKeys.MEMORY_CONSUMER$.MODULE$, consumerToSpill)); + MDC.of(LogKeys.MEMORY_CONSUMER, consumerToSpill)); throw new RuntimeException(e.getMessage()); } catch (IOException e) { logger.error("Error while calling spill() on {}", e, - MDC.of(LogKeys.MEMORY_CONSUMER$.MODULE$, consumerToSpill)); + MDC.of(LogKeys.MEMORY_CONSUMER, consumerToSpill)); // checkstyle.off: RegexpSinglelineJava throw new SparkOutOfMemoryError( "SPILL_OUT_OF_MEMORY", @@ -320,7 +320,7 @@ public void releaseExecutionMemory(long size, MemoryConsumer consumer) { */ public void showMemoryUsage() { logger.info("Memory used in task {}", - MDC.of(LogKeys.TASK_ATTEMPT_ID$.MODULE$, taskAttemptId)); + MDC.of(LogKeys.TASK_ATTEMPT_ID, taskAttemptId)); synchronized (this) { long memoryAccountedForByConsumers = 0; for (MemoryConsumer c: consumers) { @@ -328,20 +328,20 @@ public void showMemoryUsage() { memoryAccountedForByConsumers += totalMemUsage; if (totalMemUsage > 0) { logger.info("Acquired by {}: {}", - MDC.of(LogKeys.MEMORY_CONSUMER$.MODULE$, c), - MDC.of(LogKeys.MEMORY_SIZE$.MODULE$, Utils.bytesToString(totalMemUsage))); + MDC.of(LogKeys.MEMORY_CONSUMER, c), + MDC.of(LogKeys.MEMORY_SIZE, Utils.bytesToString(totalMemUsage))); } } long memoryNotAccountedFor = memoryManager.getExecutionMemoryUsageForTask(taskAttemptId) - memoryAccountedForByConsumers; logger.info( "{} bytes of memory were used by task {} but are not associated with specific consumers", - MDC.of(LogKeys.MEMORY_SIZE$.MODULE$, memoryNotAccountedFor), - MDC.of(LogKeys.TASK_ATTEMPT_ID$.MODULE$, taskAttemptId)); + MDC.of(LogKeys.MEMORY_SIZE, memoryNotAccountedFor), + MDC.of(LogKeys.TASK_ATTEMPT_ID, taskAttemptId)); logger.info( "{} bytes of memory are used for execution and {} bytes of memory are used for storage", - MDC.of(LogKeys.EXECUTION_MEMORY_SIZE$.MODULE$, memoryManager.executionMemoryUsed()), - MDC.of(LogKeys.STORAGE_MEMORY_SIZE$.MODULE$, memoryManager.storageMemoryUsed())); + MDC.of(LogKeys.EXECUTION_MEMORY_SIZE, memoryManager.executionMemoryUsed()), + MDC.of(LogKeys.STORAGE_MEMORY_SIZE, memoryManager.storageMemoryUsed())); } } @@ -388,7 +388,7 @@ public MemoryBlock allocatePage(long size, MemoryConsumer consumer) { page = memoryManager.tungstenMemoryAllocator().allocate(acquired); } catch (OutOfMemoryError e) { logger.warn("Failed to allocate a page ({} bytes), try again.", - MDC.of(LogKeys.PAGE_SIZE$.MODULE$, acquired)); + MDC.of(LogKeys.PAGE_SIZE, acquired)); // there is no enough memory actually, it means the actual free memory is smaller than // MemoryManager thought, we should keep the acquired memory. synchronized (this) { diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index 86f7d5143eff5..8072a432ab110 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -227,7 +227,7 @@ private long[] writePartitionedData(ShuffleMapOutputWriter mapOutputWriter) thro } if (!file.delete()) { logger.error("Unable to delete file for partition {}", - MDC.of(LogKeys.PARTITION_ID$.MODULE$, i)); + MDC.of(LogKeys.PARTITION_ID, i)); } } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index 7502df9e16a84..bed241b1e03a9 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -170,11 +170,11 @@ private void writeSortedFile(boolean isFinalFile) { if (!isFinalFile) { logger.info( "Task {} on Thread {} spilling sort data of {} to disk ({} {} so far)", - MDC.of(LogKeys.TASK_ATTEMPT_ID$.MODULE$, taskContext.taskAttemptId()), - MDC.of(LogKeys.THREAD_ID$.MODULE$, Thread.currentThread().getId()), - MDC.of(LogKeys.MEMORY_SIZE$.MODULE$, Utils.bytesToString(getMemoryUsage())), - MDC.of(LogKeys.NUM_SPILLS$.MODULE$, spills.size()), - MDC.of(LogKeys.SPILL_TIMES$.MODULE$, spills.size() != 1 ? "times" : "time")); + MDC.of(LogKeys.TASK_ATTEMPT_ID, taskContext.taskAttemptId()), + MDC.of(LogKeys.THREAD_ID, Thread.currentThread().getId()), + MDC.of(LogKeys.MEMORY_SIZE, Utils.bytesToString(getMemoryUsage())), + MDC.of(LogKeys.NUM_SPILLS, spills.size()), + MDC.of(LogKeys.SPILL_TIMES, spills.size() != 1 ? "times" : "time")); } // This call performs the actual sort. @@ -362,7 +362,7 @@ public void cleanupResources() { for (SpillInfo spill : spills) { if (spill.file.exists() && !spill.file.delete()) { logger.error("Unable to delete spill file {}", - MDC.of(LogKeys.PATH$.MODULE$, spill.file.getPath())); + MDC.of(LogKeys.PATH, spill.file.getPath())); } } } @@ -429,13 +429,13 @@ public void insertRecord(Object recordBase, long recordOffset, int length, int p assert(inMemSorter != null); if (inMemSorter.numRecords() >= numElementsForSpillThreshold) { logger.info("Spilling data because number of spilledRecords ({}) crossed the threshold {}", - MDC.of(LogKeys.NUM_ELEMENTS_SPILL_RECORDS$.MODULE$, inMemSorter.numRecords()), - MDC.of(LogKeys.NUM_ELEMENTS_SPILL_THRESHOLD$.MODULE$, numElementsForSpillThreshold)); + MDC.of(LogKeys.NUM_ELEMENTS_SPILL_RECORDS, inMemSorter.numRecords()), + MDC.of(LogKeys.NUM_ELEMENTS_SPILL_THRESHOLD, numElementsForSpillThreshold)); spill(); } else if (inMemRecordsSize >= recordsSizeForSpillThreshold) { logger.info("Spilling data because size of spilledRecords ({}) crossed the size threshold {}", - MDC.of(LogKeys.SPILL_RECORDS_SIZE$.MODULE$, inMemRecordsSize), - MDC.of(LogKeys.SPILL_RECORDS_SIZE_THRESHOLD$.MODULE$, recordsSizeForSpillThreshold)); + MDC.of(LogKeys.SPILL_RECORDS_SIZE, inMemRecordsSize), + MDC.of(LogKeys.SPILL_RECORDS_SIZE_THRESHOLD, recordsSizeForSpillThreshold)); spill(); } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index ac9d335d63591..e725df593a826 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -229,7 +229,7 @@ void closeAndWriteOutput() throws IOException { for (SpillInfo spill : spills) { if (spill.file.exists() && !spill.file.delete()) { logger.error("Error while deleting spill file {}", - MDC.of(LogKeys.PATH$.MODULE$, spill.file.getPath())); + MDC.of(LogKeys.PATH, spill.file.getPath())); } } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java index c0b9018c770a0..e51a48b019854 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java @@ -125,7 +125,7 @@ public void abort(Throwable error) throws IOException { cleanUp(); if (outputTempFile != null && outputTempFile.exists() && !outputTempFile.delete()) { log.warn("Failed to delete temporary shuffle file at {}", - MDC.of(LogKeys.PATH$.MODULE$, outputTempFile.getAbsolutePath())); + MDC.of(LogKeys.PATH, outputTempFile.getAbsolutePath())); } } diff --git a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java index 2a8e15cd09ccf..486bfd62bc97a 100644 --- a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java +++ b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java @@ -395,7 +395,7 @@ private void handleFailedDelete() { File file = spillWriters.removeFirst().getFile(); if (file != null && file.exists() && !file.delete()) { logger.error("Was unable to delete spill file {}", - MDC.of(LogKeys.PATH$.MODULE$, file.getAbsolutePath())); + MDC.of(LogKeys.PATH, file.getAbsolutePath())); } } } @@ -897,7 +897,7 @@ public void free() { if (file != null && file.exists()) { if (!file.delete()) { logger.error("Was unable to delete spill file {}", - MDC.of(LogKeys.PATH$.MODULE$, file.getAbsolutePath())); + MDC.of(LogKeys.PATH, file.getAbsolutePath())); } } } diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index a18e02c1974ec..94c37e187131f 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -230,10 +230,10 @@ public long spill(long size, MemoryConsumer trigger) throws IOException { } logger.info("Thread {} spilling sort data of {} to disk ({} {} so far)", - MDC.of(LogKeys.THREAD_ID$.MODULE$, Thread.currentThread().getId()), - MDC.of(LogKeys.MEMORY_SIZE$.MODULE$, Utils.bytesToString(getMemoryUsage())), - MDC.of(LogKeys.NUM_SPILL_WRITERS$.MODULE$, spillWriters.size()), - MDC.of(LogKeys.SPILL_TIMES$.MODULE$, spillWriters.size() > 1 ? "times" : "time")); + MDC.of(LogKeys.THREAD_ID, Thread.currentThread().getId()), + MDC.of(LogKeys.MEMORY_SIZE, Utils.bytesToString(getMemoryUsage())), + MDC.of(LogKeys.NUM_SPILL_WRITERS, spillWriters.size()), + MDC.of(LogKeys.SPILL_TIMES, spillWriters.size() > 1 ? "times" : "time")); ShuffleWriteMetrics writeMetrics = new ShuffleWriteMetrics(); @@ -350,7 +350,7 @@ private void deleteSpillFiles() { if (file != null && file.exists()) { if (!file.delete()) { logger.error("Was unable to delete spill file {}", - MDC.of(LogKeys.PATH$.MODULE$, file.getAbsolutePath())); + MDC.of(LogKeys.PATH, file.getAbsolutePath())); } } } @@ -492,13 +492,13 @@ public void insertRecord( assert(inMemSorter != null); if (inMemSorter.numRecords() >= numElementsForSpillThreshold) { logger.info("Spilling data because number of spilledRecords ({}) crossed the threshold {}", - MDC.of(LogKeys.NUM_ELEMENTS_SPILL_RECORDS$.MODULE$, inMemSorter.numRecords()), - MDC.of(LogKeys.NUM_ELEMENTS_SPILL_THRESHOLD$.MODULE$, numElementsForSpillThreshold)); + MDC.of(LogKeys.NUM_ELEMENTS_SPILL_RECORDS, inMemSorter.numRecords()), + MDC.of(LogKeys.NUM_ELEMENTS_SPILL_THRESHOLD, numElementsForSpillThreshold)); spill(); } else if (inMemRecordsSize >= recordsSizeForSpillThreshold) { logger.info("Spilling data because size of spilledRecords ({}) crossed the size threshold {}", - MDC.of(LogKeys.SPILL_RECORDS_SIZE$.MODULE$, inMemRecordsSize), - MDC.of(LogKeys.SPILL_RECORDS_SIZE_THRESHOLD$.MODULE$, recordsSizeForSpillThreshold)); + MDC.of(LogKeys.SPILL_RECORDS_SIZE, inMemRecordsSize), + MDC.of(LogKeys.SPILL_RECORDS_SIZE_THRESHOLD, recordsSizeForSpillThreshold)); spill(); } diff --git a/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala b/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala index 3f95515c04d29..2fdb8debf1909 100644 --- a/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala @@ -24,7 +24,7 @@ import java.util.function.Consumer import scala.collection.mutable.{ArrayBuffer, HashSet} import scala.jdk.CollectionConverters._ -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.rpc.{RpcCallContext, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.scheduler.{LiveListenerBus, SparkListener, SparkListenerStageCompleted} diff --git a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala index 47f287293974f..ee1e4dafcfc1a 100644 --- a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala +++ b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala @@ -27,7 +27,7 @@ import scala.util.{Failure, Success => ScalaSuccess, Try} import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.executor.TaskMetrics -import org.apache.spark.internal.{Logging, MDC, MessageWithContext} +import org.apache.spark.internal.{Logging, MessageWithContext} import org.apache.spark.internal.LogKeys._ import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.metrics.source.Source diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index fb56389cde77e..54ea8c94daac1 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -24,7 +24,7 @@ import java.util.concurrent.{ConcurrentHashMap, ConcurrentLinkedQueue, Scheduled import scala.jdk.CollectionConverters._ import org.apache.spark.broadcast.Broadcast -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{ACCUMULATOR_ID, BROADCAST_ID, LISTENER, RDD_ID, SHUFFLE_ID} import org.apache.spark.internal.config._ import org.apache.spark.rdd.{RDD, ReliableRDDCheckpointData} diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 573608c4327e0..745faf866cebf 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -24,7 +24,7 @@ import scala.reflect.ClassTag import org.roaringbitmap.RoaringBitmap import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 3d0b58b429116..16ec7689d5261 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -26,7 +26,7 @@ import scala.util.control.NonFatal import com.codahale.metrics.{Counter, Gauge, MetricRegistry} -import org.apache.spark.internal.{config, Logging, MDC} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config._ import org.apache.spark.internal.config.DECOMMISSION_ENABLED diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 92aea5959aab7..40ea5c59ec04e 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -23,7 +23,7 @@ import scala.collection.mutable.{HashMap, Map} import scala.concurrent.Future import org.apache.spark.executor.ExecutorMetrics -import org.apache.spark.internal.{config, Logging, MDC} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config.Network import org.apache.spark.rpc.{IsolatedThreadSafeRpcEndpoint, RpcCallContext, RpcEnv} diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index a660bccd2e68f..9b2d3d748ed4d 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -34,7 +34,7 @@ import org.apache.commons.io.output.{ByteArrayOutputStream => ApacheByteArrayOut import org.roaringbitmap.RoaringBitmap import org.apache.spark.broadcast.{Broadcast, BroadcastManager} -import org.apache.spark.internal.{Logging, MDC, MessageWithContext} +import org.apache.spark.internal.{Logging, MessageWithContext} import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config._ import org.apache.spark.io.CompressionCodec diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index c951876e62034..1461677219bc1 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.io.Text import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.config._ import org.apache.spark.internal.config.UI._ import org.apache.spark.launcher.SparkLauncher diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index bbf2809b7db23..1d1901ae4f688 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -25,7 +25,7 @@ import scala.jdk.CollectionConverters._ import org.apache.avro.{Schema, SchemaNormalization} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys import org.apache.spark.internal.config._ import org.apache.spark.internal.config.History._ diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 23dda06326d70..ce62f8f5d5473 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -46,7 +46,7 @@ import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import org.apache.spark.errors.SparkCoreErrors import org.apache.spark.executor.{Executor, ExecutorMetrics, ExecutorMetricsSource} import org.apache.spark.input.{FixedLengthBinaryInputFormat, PortableDataStream, StreamInputFormat, WholeTextFileInputFormat} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Tests._ diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index bf6e30f5afade..796dbf4b6d5f8 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -32,7 +32,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.python.{PythonWorker, PythonWorkerFactory} import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.executor.ExecutorBackend -import org.apache.spark.internal.{config, Logging, MDC} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.LogKeys import org.apache.spark.internal.config._ import org.apache.spark.memory.{MemoryManager, UnifiedMemoryManager} diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index f0e844289b9db..a7f09028597af 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -25,7 +25,7 @@ import scala.collection.mutable.ArrayBuffer import scala.jdk.CollectionConverters._ import org.apache.spark.executor.TaskMetrics -import org.apache.spark.internal.{config, Logging, MDC} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.LogKeys.LISTENER import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.metrics.MetricsSystem diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonGatewayServer.scala b/core/src/main/scala/org/apache/spark/api/python/PythonGatewayServer.scala index 7737822f2af2b..a1c11a8da9323 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonGatewayServer.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonGatewayServer.scala @@ -23,7 +23,7 @@ import java.nio.charset.StandardCharsets.UTF_8 import java.nio.file.Files import org.apache.spark.SparkConf -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CLASS_NAME, PATH} /** diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala index 5e2b5553f3dca..d41ac4603a636 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.io._ import org.apache.spark.SparkException import org.apache.spark.broadcast.Broadcast -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.CLASS_NAME import org.apache.spark.rdd.RDD import org.apache.spark.util.{SerializableConfiguration, Utils} diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index f2b6b8c1e9f8d..cf0169fed60c4 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -38,7 +38,7 @@ import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} import org.apache.spark.api.python.PythonFunction.PythonAccumulator import org.apache.spark.broadcast.Broadcast import org.apache.spark.input.PortableDataStream -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{HOST, PORT, SOCKET_ADDRESS} import org.apache.spark.internal.config.BUFFER_SIZE import org.apache.spark.internal.config.Python.PYTHON_UNIX_DOMAIN_SOCKET_ENABLED diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala index 6d2d81fdf8178..ca4f6e56554ec 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala @@ -32,7 +32,7 @@ import scala.util.control.NonFatal import org.apache.spark._ import org.apache.spark.api.python.PythonFunction.PythonAccumulator -import org.apache.spark.internal.{Logging, LogKeys, MDC, MessageWithContext} +import org.apache.spark.internal.{Logging, LogKeys, MessageWithContext} import org.apache.spark.internal.LogKeys.TASK_NAME import org.apache.spark.internal.config.{BUFFER_SIZE, EXECUTOR_CORES} import org.apache.spark.internal.config.Python._ diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala index 816ceea327aae..52ad6a93ce4c6 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala @@ -28,7 +28,7 @@ import scala.sys.process.Process import org.apache.spark.{SparkContext, SparkEnv} import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{PATH, PYTHON_PACKAGES, PYTHON_VERSION} import org.apache.spark.util.ArrayImplicits.SparkArrayOps import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index a16090ce6276f..bbce14a207492 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -32,7 +32,7 @@ import scala.jdk.OptionConverters._ import org.apache.spark._ import org.apache.spark.errors.SparkCoreErrors -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config.Python.{PYTHON_UNIX_DOMAIN_SOCKET_DIR, PYTHON_UNIX_DOMAIN_SOCKET_ENABLED} import org.apache.spark.security.SocketAuthHelper diff --git a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala index 0fe57dd0bb0ae..e6f2a68567648 100644 --- a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala @@ -28,7 +28,7 @@ import net.razorvine.pickle.{Pickler, Unpickler} import org.apache.spark.SparkException import org.apache.spark.api.java.JavaRDD -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.rdd.RDD import org.apache.spark.util.ArrayImplicits._ diff --git a/core/src/main/scala/org/apache/spark/api/python/StreamingPythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/StreamingPythonRunner.scala index 7eba574751b46..e61fa01db113b 100644 --- a/core/src/main/scala/org/apache/spark/api/python/StreamingPythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/StreamingPythonRunner.scala @@ -23,7 +23,7 @@ import java.nio.channels.Channels import scala.jdk.CollectionConverters._ import org.apache.spark.{SparkEnv, SparkPythonException} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{PYTHON_WORKER_MODULE, PYTHON_WORKER_RESPONSE, SESSION_ID} import org.apache.spark.internal.config.BUFFER_SIZE import org.apache.spark.internal.config.Python.{PYTHON_AUTH_SOCKET_TIMEOUT, PYTHON_UNIX_DOMAIN_SOCKET_ENABLED} diff --git a/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala index c3d01ec47458e..622833c9cd6af 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala @@ -26,7 +26,7 @@ import io.netty.handler.timeout.ReadTimeoutException import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.api.r.SerDe._ -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config.R._ import org.apache.spark.util.{ThreadUtils, Utils} diff --git a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala index 3adb540a7ad18..ad1a274170a2a 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala @@ -22,7 +22,7 @@ import java.io.Serializable import scala.reflect.ClassTag import org.apache.spark.SparkException -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.util.Utils /** diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 0c7ec5c1a98a7..0f93db9b96b58 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -27,7 +27,7 @@ import scala.reflect.ClassTag import scala.util.Random import org.apache.spark._ -import org.apache.spark.internal.{config, Logging, MDC} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.LogKeys._ import org.apache.spark.io.CompressionCodec import org.apache.spark.serializer.Serializer diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index 226a6dcd36a16..2891247f20104 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -31,7 +31,7 @@ import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.{DriverState, Master} import org.apache.spark.deploy.master.DriverState.DriverState -import org.apache.spark.internal.{config, Logging, MDC} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config.Network.RPC_ASK_TIMEOUT import org.apache.spark.resource.ResourceUtils diff --git a/core/src/main/scala/org/apache/spark/deploy/DriverTimeoutPlugin.scala b/core/src/main/scala/org/apache/spark/deploy/DriverTimeoutPlugin.scala index 736c23556ec15..c1db3ddebc8ca 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DriverTimeoutPlugin.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DriverTimeoutPlugin.scala @@ -23,7 +23,7 @@ import scala.jdk.CollectionConverters._ import org.apache.spark.SparkContext import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config.DRIVER_TIMEOUT import org.apache.spark.util.{SparkExitCode, ThreadUtils} diff --git a/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala b/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala index e21c772c00779..fec69aed48981 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala @@ -23,7 +23,7 @@ import java.util.concurrent.CountDownLatch import scala.jdk.CollectionConverters._ import org.apache.spark.{SecurityManager, SparkConf} -import org.apache.spark.internal.{config, Logging, MDC} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.LogKeys.{AUTH_ENABLED, PORT, SHUFFLE_DB_BACKEND_KEY, SHUFFLE_DB_BACKEND_NAME} import org.apache.spark.metrics.{MetricsSystem, MetricsSystemInstances} import org.apache.spark.network.TransportContext diff --git a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala index 263b1a233b808..e03dc2ff00a08 100644 --- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala @@ -24,7 +24,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.SparkConf import org.apache.spark.deploy.master.Master import org.apache.spark.deploy.worker.Worker -import org.apache.spark.internal.{config, Logging, LogKeys, MDC} +import org.apache.spark.internal.{config, Logging, LogKeys} import org.apache.spark.rpc.RpcEnv import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/deploy/RPackageUtils.scala b/core/src/main/scala/org/apache/spark/deploy/RPackageUtils.scala index ac2c1f73bd096..d315155ec44a7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/RPackageUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/RPackageUtils.scala @@ -27,7 +27,7 @@ import scala.jdk.CollectionConverters._ import com.google.common.io.{ByteStreams, Files} import org.apache.spark.api.r.RUtils -import org.apache.spark.internal.{LogEntry, Logging, MDC, MessageWithContext} +import org.apache.spark.internal.{LogEntry, Logging, MessageWithContext} import org.apache.spark.internal.LogKeys._ import org.apache.spark.util.{RedirectThread, Utils} diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index ca932ef5dc05c..9c40f4cd1cf75 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -37,7 +37,7 @@ import org.apache.hadoop.security.token.{Token, TokenIdentifier} import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.config.BUFFER_SIZE import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 39cb6dfc4c84d..7029980cdb440 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -38,7 +38,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.spark._ import org.apache.spark.api.r.RUtils import org.apache.spark.deploy.rest._ -import org.apache.spark.internal.{LogEntry, Logging, LogKeys, MDC} +import org.apache.spark.internal.{LogEntry, Logging, LogKeys} import org.apache.spark.internal.config._ import org.apache.spark.internal.config.UI._ import org.apache.spark.launcher.SparkLauncher diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index f4884385555f6..3eb2fd2a0e4d2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -28,7 +28,7 @@ import scala.util.Try import org.apache.spark.{SparkConf, SparkException, SparkUserAppException} import org.apache.spark.deploy.SparkSubmitAction._ -import org.apache.spark.internal.{config, Logging, MDC} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config.DYN_ALLOCATION_ENABLED import org.apache.spark.launcher.SparkSubmitArgumentsParser diff --git a/core/src/main/scala/org/apache/spark/deploy/StandaloneResourceUtils.scala b/core/src/main/scala/org/apache/spark/deploy/StandaloneResourceUtils.scala index 7b98461b01acf..5a1d58897703d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/StandaloneResourceUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/StandaloneResourceUtils.scala @@ -27,7 +27,7 @@ import org.json4s.{DefaultFormats, Extraction, Formats} import org.json4s.jackson.JsonMethods.{compact, render} import org.apache.spark.SparkException -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.COMPONENT import org.apache.spark.resource.{ResourceAllocation, ResourceID, ResourceInformation, ResourceRequirement} import org.apache.spark.util.ArrayImplicits._ diff --git a/core/src/main/scala/org/apache/spark/deploy/Utils.scala b/core/src/main/scala/org/apache/spark/deploy/Utils.scala index b3d871d75e6c7..9ff1333b03e62 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Utils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Utils.scala @@ -22,7 +22,7 @@ import java.io.File import jakarta.servlet.http.HttpServletRequest import org.apache.spark.SparkConf -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{LOG_TYPE, PATH} import org.apache.spark.ui.JettyUtils.createServletHandler import org.apache.spark.ui.WebUI diff --git a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala index b34e5c408c3be..2d742b31f99c2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala @@ -29,7 +29,7 @@ import org.apache.spark.SparkConf import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.LogKeys._ import org.apache.spark.resource.ResourceProfile import org.apache.spark.rpc._ diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala index 8caf67ff4680b..e979671890eac 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala @@ -28,7 +28,7 @@ import jakarta.servlet.{DispatcherType, Filter, FilterChain, ServletException, S import jakarta.servlet.http.{HttpServletRequest, HttpServletResponse} import org.eclipse.jetty.servlet.FilterHolder -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.metrics.source.Source import org.apache.spark.ui.SparkUI diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala index 20e60c679b06a..9a983ae3a5d43 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala @@ -23,7 +23,7 @@ import scala.util.control.NonFatal import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.deploy.history.EventFilter.FilterStatistics -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{LINE, LINE_NUM, PATH} import org.apache.spark.scheduler._ import org.apache.spark.util.{JsonProtocol, Utils} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala index 5cec3d2a9d28f..7890909854530 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala @@ -28,7 +28,7 @@ import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.spark.SparkConf import org.apache.spark.deploy.history.EventFilter.FilterStatistics -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys import org.apache.spark.scheduler.ReplayListenerBus import org.apache.spark.util.{JsonProtocol, Utils} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala index 83bc1443112a0..4e3bee1015ff3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala @@ -28,7 +28,7 @@ import org.apache.hadoop.fs.permission.FsPermission import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config._ diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 4365e92280723..356c388dc7700 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -37,7 +37,7 @@ import org.apache.hadoop.security.AccessControlException import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config._ diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index aba66c319ca9d..14daa5d88b1c4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -28,7 +28,7 @@ import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.Utils.addRenderLogHandler -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config.History import org.apache.spark.internal.config.UI._ diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala index 8583b0fa1625e..753fe59dc0894 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala @@ -23,7 +23,7 @@ import java.util.concurrent.atomic.AtomicLong import scala.collection.mutable.{HashMap, ListBuffer} import org.apache.spark.SparkConf -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config.History import org.apache.spark.internal.config.History._ diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerMemoryManager.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerMemoryManager.scala index 6e3dbb1170998..f9a6c7f64896c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerMemoryManager.scala @@ -22,7 +22,7 @@ import java.util.concurrent.atomic.AtomicLong import scala.collection.mutable.HashMap import org.apache.spark.SparkConf -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config.History._ import org.apache.spark.io.CompressionCodec diff --git a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala index 4332544e4491c..0c9baf9e6f70c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala @@ -22,7 +22,7 @@ import java.nio.file.{FileAlreadyExistsException, Files, Paths} import scala.reflect.ClassTag -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.io.CompressionCodec import org.apache.spark.serializer.{DeserializationStream, SerializationStream, Serializer} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 7d15744de6b45..95778771d4239 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -32,7 +32,7 @@ import org.apache.spark.deploy.master.DriverState.DriverState import org.apache.spark.deploy.master.MasterMessages._ import org.apache.spark.deploy.master.ui.MasterWebUI import org.apache.spark.deploy.rest.StandaloneRestServer -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Deploy._ diff --git a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala index 964b115865aef..a751c7bfc67c4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.master import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.config.Deploy.{RECOVERY_COMPRESSION_CODEC, RECOVERY_DIRECTORY} import org.apache.spark.io.CompressionCodec import org.apache.spark.serializer.Serializer diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index 6c7a8f582d915..66036e7a5e5ce 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -24,7 +24,7 @@ import jakarta.servlet.http.{HttpServlet, HttpServletRequest, HttpServletRespons import org.apache.spark.deploy.DeployMessages.{DecommissionWorkersOnHosts, MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.Utils.addRenderLogHandler import org.apache.spark.deploy.master.Master -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{HOSTS, NUM_REMOVED_WORKERS} import org.apache.spark.internal.config.DECOMMISSION_ENABLED import org.apache.spark.internal.config.UI.MASTER_UI_DECOMMISSION_ALLOW_MODE diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala index 3c0baacbf10be..6dc422515dbdb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala @@ -33,7 +33,7 @@ import jakarta.servlet.http.HttpServletResponse import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf, SparkException} import org.apache.spark.deploy.SparkApplication -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala index f8afa86fd36fc..e172a06f0a32e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala @@ -32,7 +32,7 @@ import org.json4s._ import org.json4s.jackson.JsonMethods._ import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config.{MASTER_REST_SERVER_FILTERS, MASTER_REST_SERVER_MAX_THREADS, MASTER_REST_SERVER_VIRTUAL_THREADS} import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HBaseDelegationTokenProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HBaseDelegationTokenProvider.scala index b3d67028e4170..83f36e1245844 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HBaseDelegationTokenProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HBaseDelegationTokenProvider.scala @@ -27,7 +27,7 @@ import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.token.{Token, TokenIdentifier} import org.apache.spark.SparkConf -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.security.HadoopDelegationTokenProvider import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala index b9d88266ed538..bc77159415473 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala @@ -31,7 +31,7 @@ import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys import org.apache.spark.internal.config._ import org.apache.spark.rpc.RpcEndpointRef diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala index fc750b54d0b8e..3a38eb4a08d7d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala @@ -28,7 +28,7 @@ import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config._ import org.apache.spark.security.HadoopDelegationTokenProvider diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index a3e7276fc83e1..5bc17e2164738 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -24,7 +24,7 @@ import scala.jdk.CollectionConverters._ import org.apache.spark.{SecurityManager, SSLOptions} import org.apache.spark.deploy.Command -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.launcher.WorkerCommandBuilder import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index e52bb3ae6d22e..32ed6f2c47dc9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -31,7 +31,7 @@ import org.apache.spark.deploy.DeployMessages.DriverStateChanged import org.apache.spark.deploy.StandaloneResourceUtils.prepareResourcesFile import org.apache.spark.deploy.master.DriverState import org.apache.spark.deploy.master.DriverState.DriverState -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config.{DRIVER_RESOURCES_FILE, SPARK_DRIVER_PREFIX} import org.apache.spark.internal.config.UI.UI_REVERSE_PROXY diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala index 4f42088903464..1d64dbc224c49 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala @@ -21,7 +21,7 @@ import java.io.File import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.internal.{config, Logging, MDC} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.LogKeys.RPC_ADDRESS import org.apache.spark.rpc.RpcEnv import org.apache.spark.util._ diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 1476bdd5c016a..653d24f0ff0c7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -28,7 +28,7 @@ import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged import org.apache.spark.deploy.StandaloneResourceUtils.prepareResourcesFile -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config.SPARK_EXECUTOR_PREFIX import org.apache.spark.internal.config.UI._ diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 488bc6c791311..98da33a429eca 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -37,7 +37,7 @@ import org.apache.spark.deploy.ExternalShuffleService import org.apache.spark.deploy.StandaloneResourceUtils._ import org.apache.spark.deploy.master.{DriverState, Master} import org.apache.spark.deploy.worker.ui.WorkerWebUI -import org.apache.spark.internal.{config, Logging, MDC} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.internal.config.UI._ diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala index bd07a0ade523d..ef6d84631c83c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.worker import java.util.concurrent.atomic.AtomicBoolean -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.LogKeys.WORKER_URL import org.apache.spark.rpc._ diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala index defce5acc6168..bdc143be91b94 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala @@ -23,7 +23,7 @@ import scala.xml.{Node, Unparsed} import jakarta.servlet.http.HttpServletRequest -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{LOG_TYPE, PATH} import org.apache.spark.ui.{UIUtils, WebUIPage} import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index a30759e5d794e..206a6a0fe385c 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -31,7 +31,7 @@ import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.worker.WorkerWatcher -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys import org.apache.spark.internal.config._ import org.apache.spark.network.netty.SparkTransportConf diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 4fccd2115eb38..a14ba21a0c186 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -36,11 +36,11 @@ import scala.util.control.NonFatal import com.google.common.cache.{Cache, CacheBuilder, RemovalListener, RemovalNotification} import com.google.common.util.concurrent.ThreadFactoryBuilder -import org.slf4j.MDC +import org.slf4j.{MDC => SLF4JMDC} import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.internal.{Logging, LogKeys, MDC => LogMDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config._ import org.apache.spark.internal.config.{EXECUTOR_USER_CLASS_PATH_FIRST => EXECUTOR_USER_CLASS_PATH_FIRST_CONFIG} @@ -82,12 +82,12 @@ private[spark] class Executor( resources: immutable.Map[String, ResourceInformation]) extends Logging { - logInfo(log"Starting executor ID ${LogMDC(LogKeys.EXECUTOR_ID, executorId)}" + - log" on host ${LogMDC(HOST, executorHostname)}") - logInfo(log"OS info ${LogMDC(OS_NAME, Utils.osName)}," + - log" ${LogMDC(OS_VERSION, Utils.osVersion)}, " + - log"${LogMDC(OS_ARCH, Utils.osArch)}") - logInfo(log"Java version ${LogMDC(JAVA_VERSION, Utils.javaVersion)}") + logInfo(log"Starting executor ID ${MDC(LogKeys.EXECUTOR_ID, executorId)}" + + log" on host ${MDC(HOST, executorHostname)}") + logInfo(log"OS info ${MDC(OS_NAME, Utils.osName)}," + + log" ${MDC(OS_VERSION, Utils.osVersion)}, " + + log"${MDC(OS_ARCH, Utils.osArch)}") + logInfo(log"Java version ${MDC(JAVA_VERSION, Utils.javaVersion)}") private val executorShutdown = new AtomicBoolean(false) val stopHookReference = ShutdownHookManager.addShutdownHook( @@ -102,7 +102,7 @@ private[spark] class Executor( private[executor] val taskNameMDCKey = if (conf.get(LEGACY_TASK_NAME_MDC_ENABLED)) { "mdc.taskName" } else { - LogKeys.TASK_NAME.name + TASK_NAME.name.toLowerCase(Locale.ROOT) } // SPARK-40235: updateDependencies() uses a ReentrantLock instead of the `synchronized` keyword @@ -223,7 +223,7 @@ private[spark] class Executor( if (sessionBasedRoot.isDirectory && sessionBasedRoot.exists()) { Utils.deleteRecursively(sessionBasedRoot) } - logInfo(log"Session evicted: ${LogMDC(SESSION_ID, state.sessionUUID)}") + logInfo(log"Session evicted: ${MDC(SESSION_ID, state.sessionUUID)}") } }) .build[String, IsolatedSessionState] @@ -505,9 +505,9 @@ private[spark] class Executor( @volatile var task: Task[Any] = _ def kill(interruptThread: Boolean, reason: String): Unit = { - logInfo(log"Executor is trying to kill ${LogMDC(TASK_NAME, taskName)}, " + - log"interruptThread: ${LogMDC(INTERRUPT_THREAD, interruptThread)}, " + - log"reason: ${LogMDC(REASON, reason)}") + logInfo(log"Executor is trying to kill ${MDC(TASK_NAME, taskName)}, " + + log"interruptThread: ${MDC(INTERRUPT_THREAD, interruptThread)}, " + + log"reason: ${MDC(REASON, reason)}") reasonIfKilled = Some(reason) if (task != null) { synchronized { @@ -579,7 +579,7 @@ private[spark] class Executor( } else 0L Thread.currentThread.setContextClassLoader(isolatedSession.replClassLoader) val ser = env.closureSerializer.newInstance() - logInfo(log"Running ${LogMDC(TASK_NAME, taskName)}") + logInfo(log"Running ${MDC(TASK_NAME, taskName)}") execBackend.statusUpdate(taskId, TaskState.RUNNING, EMPTY_BYTE_BUFFER) var taskStartTimeNs: Long = 0 var taskStartCpu: Long = 0 @@ -653,7 +653,7 @@ private[spark] class Executor( if (freedMemory > 0 && !threwException) { val errMsg = log"Managed memory leak detected; size = " + - log"${LogMDC(NUM_BYTES, freedMemory)} bytes, ${LogMDC(TASK_NAME, taskName)}" + log"${MDC(NUM_BYTES, freedMemory)} bytes, ${MDC(TASK_NAME, taskName)}" if (conf.get(UNSAFE_EXCEPTION_ON_MEMORY_LEAK)) { throw SparkException.internalError(errMsg.message, category = "EXECUTOR") } else { @@ -663,9 +663,9 @@ private[spark] class Executor( if (releasedLocks.nonEmpty && !threwException) { val errMsg = - log"${LogMDC(NUM_RELEASED_LOCKS, releasedLocks.size)} block locks" + - log" were not released by ${LogMDC(TASK_NAME, taskName)}\n" + - log" ${LogMDC(RELEASED_LOCKS, releasedLocks.mkString("[", ", ", "]"))})" + log"${MDC(NUM_RELEASED_LOCKS, releasedLocks.size)} block locks" + + log" were not released by ${MDC(TASK_NAME, taskName)}\n" + + log" ${MDC(RELEASED_LOCKS, releasedLocks.mkString("[", ", ", "]"))})" if (conf.get(STORAGE_EXCEPTION_PIN_LEAK)) { throw SparkException.internalError(errMsg.message, category = "EXECUTOR") } else { @@ -677,10 +677,10 @@ private[spark] class Executor( // uh-oh. it appears the user code has caught the fetch-failure without throwing any // other exceptions. Its *possible* this is what the user meant to do (though highly // unlikely). So we will log an error and keep going. - logError(log"${LogMDC(TASK_NAME, taskName)} completed successfully though internally " + + logError(log"${MDC(TASK_NAME, taskName)} completed successfully though internally " + log"it encountered unrecoverable fetch failures! Most likely this means user code " + log"is incorrectly swallowing Spark's internal " + - log"${LogMDC(CLASS_NAME, classOf[FetchFailedException])}", fetchFailure) + log"${MDC(CLASS_NAME, classOf[FetchFailedException])}", fetchFailure) } val taskFinishNs = System.nanoTime() val taskFinishCpu = if (threadMXBean.isCurrentThreadCpuTimeSupported) { @@ -746,10 +746,10 @@ private[spark] class Executor( // directSend = sending directly back to the driver val serializedResult: ByteBuffer = { if (maxResultSize > 0 && resultSize > maxResultSize) { - logWarning(log"Finished ${LogMDC(TASK_NAME, taskName)}. " + + logWarning(log"Finished ${MDC(TASK_NAME, taskName)}. " + log"Result is larger than maxResultSize " + - log"(${LogMDC(RESULT_SIZE_BYTES, Utils.bytesToString(resultSize))} > " + - log"${LogMDC(RESULT_SIZE_BYTES_MAX, Utils.bytesToString(maxResultSize))}), " + + log"(${MDC(RESULT_SIZE_BYTES, Utils.bytesToString(resultSize))} > " + + log"${MDC(RESULT_SIZE_BYTES_MAX, Utils.bytesToString(maxResultSize))}), " + log"dropping it.") ser.serialize(new IndirectTaskResult[Any](TaskResultBlockId(taskId), resultSize)) } else if (resultSize > maxDirectResultSize) { @@ -758,12 +758,12 @@ private[spark] class Executor( blockId, serializedDirectResult, StorageLevel.MEMORY_AND_DISK_SER) - logInfo(log"Finished ${LogMDC(TASK_NAME, taskName)}." + - log" ${LogMDC(NUM_BYTES, resultSize)} bytes result sent via BlockManager)") + logInfo(log"Finished ${MDC(TASK_NAME, taskName)}." + + log" ${MDC(NUM_BYTES, resultSize)} bytes result sent via BlockManager)") ser.serialize(new IndirectTaskResult[Any](blockId, resultSize)) } else { - logInfo(log"Finished ${LogMDC(TASK_NAME, taskName)}." + - log" ${LogMDC(NUM_BYTES, resultSize)} bytes result sent to driver") + logInfo(log"Finished ${MDC(TASK_NAME, taskName)}." + + log" ${MDC(NUM_BYTES, resultSize)} bytes result sent to driver") // toByteBuffer is safe here, guarded by maxDirectResultSize serializedDirectResult.toByteBuffer } @@ -775,8 +775,8 @@ private[spark] class Executor( execBackend.statusUpdate(taskId, TaskState.FINISHED, serializedResult) } catch { case t: TaskKilledException => - logInfo(log"Executor killed ${LogMDC(TASK_NAME, taskName)}," + - log" reason: ${LogMDC(REASON, t.reason)}") + logInfo(log"Executor killed ${MDC(TASK_NAME, taskName)}," + + log" reason: ${MDC(REASON, t.reason)}") val (accums, accUpdates) = collectAccumulatorsAndResetStatusOnFailure(taskStartTimeNs) // Here and below, put task metric peaks in an immutable.ArraySeq to expose them as an @@ -789,8 +789,8 @@ private[spark] class Executor( case _: InterruptedException | NonFatal(_) if task != null && task.reasonIfKilled.isDefined => val killReason = task.reasonIfKilled.getOrElse("unknown reason") - logInfo(log"Executor interrupted and killed ${LogMDC(TASK_NAME, taskName)}," + - log" reason: ${LogMDC(REASON, killReason)}") + logInfo(log"Executor interrupted and killed ${MDC(TASK_NAME, taskName)}," + + log" reason: ${MDC(REASON, killReason)}") val (accums, accUpdates) = collectAccumulatorsAndResetStatusOnFailure(taskStartTimeNs) val metricPeaks = metricsPoller.getTaskMetricPeaks(taskId).toImmutableArraySeq @@ -803,12 +803,12 @@ private[spark] class Executor( if (!t.isInstanceOf[FetchFailedException]) { // there was a fetch failure in the task, but some user code wrapped that exception // and threw something else. Regardless, we treat it as a fetch failure. - logWarning(log"${LogMDC(TASK_NAME, taskName)} encountered a " + - log"${LogMDC(CLASS_NAME, classOf[FetchFailedException].getName)} " + + logWarning(log"${MDC(TASK_NAME, taskName)} encountered a " + + log"${MDC(CLASS_NAME, classOf[FetchFailedException].getName)} " + log"and failed, but the " + - log"${LogMDC(CLASS_NAME, classOf[FetchFailedException].getName)} " + + log"${MDC(CLASS_NAME, classOf[FetchFailedException].getName)} " + log"was hidden by another exception. Spark is handling this like a fetch failure " + - log"and ignoring the other exception: ${LogMDC(ERROR, t)}") + log"and ignoring the other exception: ${MDC(ERROR, t)}") } setTaskFinishedAndClearInterruptStatus() plugins.foreach(_.onTaskFailed(reason)) @@ -823,13 +823,13 @@ private[spark] class Executor( case t: Throwable if env.isStopped => // Log the expected exception after executor.stop without stack traces // see: SPARK-19147 - logError(log"Exception in ${LogMDC(TASK_NAME, taskName)}: ${LogMDC(ERROR, t.getMessage)}") + logError(log"Exception in ${MDC(TASK_NAME, taskName)}: ${MDC(ERROR, t.getMessage)}") case t: Throwable => // Attempt to exit cleanly by informing the driver of our failure. // If anything goes wrong (or this was a fatal exception), we will delegate to // the default uncaught exception handler, which will terminate the Executor. - logError(log"Exception in ${LogMDC(TASK_NAME, taskName)}", t) + logError(log"Exception in ${MDC(TASK_NAME, taskName)}", t) // SPARK-20904: Do not report failure to driver if if happened during shut down. Because // libraries may set up shutdown hooks that race with running tasks during shutdown, @@ -934,16 +934,16 @@ private[spark] class Executor( private def setMDCForTask(taskName: String, mdc: Seq[(String, String)]): Unit = { if (Executor.mdcIsSupported) { - mdc.foreach { case (key, value) => MDC.put(key, value) } + mdc.foreach { case (key, value) => SLF4JMDC.put(key, value) } // avoid overriding the takName by the user - MDC.put(taskNameMDCKey, taskName) + SLF4JMDC.put(taskNameMDCKey, taskName) } } private def cleanMDCForTask(taskName: String, mdc: Seq[(String, String)]): Unit = { if (Executor.mdcIsSupported) { - mdc.foreach { case (key, _) => MDC.remove(key) } - MDC.remove(taskNameMDCKey) + mdc.foreach { case (key, _) => SLF4JMDC.remove(key) } + SLF4JMDC.remove(taskNameMDCKey) } } @@ -1016,14 +1016,14 @@ private[spark] class Executor( finished = true } else { val elapsedTimeMs = TimeUnit.NANOSECONDS.toMillis(elapsedTimeNs) - logWarning(log"Killed task ${LogMDC(TASK_ID, taskId)} " + - log"is still running after ${LogMDC(TIME_UNITS, elapsedTimeMs)} ms") + logWarning(log"Killed task ${MDC(TASK_ID, taskId)} " + + log"is still running after ${MDC(TIME_UNITS, elapsedTimeMs)} ms") if (takeThreadDump) { try { taskRunner.theadDump().foreach { thread => if (thread.threadName == taskRunner.threadName) { - logWarning(log"Thread dump from task ${LogMDC(TASK_ID, taskId)}:\n" + - log"${LogMDC(THREAD, thread.toString)}") + logWarning(log"Thread dump from task ${MDC(TASK_ID, taskId)}:\n" + + log"${MDC(THREAD, thread.toString)}") } } } catch { @@ -1037,8 +1037,8 @@ private[spark] class Executor( if (!taskRunner.isFinished && timeoutExceeded()) { val killTimeoutMs = TimeUnit.NANOSECONDS.toMillis(killTimeoutNs) if (isLocal) { - logError(log"Killed task ${LogMDC(TASK_ID, taskId)} could not be stopped within " + - log"${LogMDC(TIMEOUT, killTimeoutMs)} ms; " + + logError(log"Killed task ${MDC(TASK_ID, taskId)} could not be stopped within " + + log"${MDC(TIMEOUT, killTimeoutMs)} ms; " + log"not killing JVM because we are running in local mode.") } else { // In non-local-mode, the exception thrown here will bubble up to the uncaught exception @@ -1093,8 +1093,8 @@ private[spark] class Executor( logInfo( log"Starting executor with user classpath" + log" (userClassPathFirst =" + - log" ${LogMDC(LogKeys.EXECUTOR_USER_CLASS_PATH_FIRST, userClassPathFirst)}): " + - log"${LogMDC(URLS, urls.mkString("'", ",", "'"))}" + log" ${MDC(LogKeys.EXECUTOR_USER_CLASS_PATH_FIRST, userClassPathFirst)}): " + + log"${MDC(URLS, urls.mkString("'", ",", "'"))}" ) if (useStub) { @@ -1150,13 +1150,13 @@ private[spark] class Executor( sessionUUID: String): ClassLoader = { val classUri = sessionClassUri.getOrElse(conf.get("spark.repl.class.uri", null)) val classLoader = if (classUri != null) { - logInfo(log"Using REPL class URI: ${LogMDC(LogKeys.URI, classUri)}") + logInfo(log"Using REPL class URI: ${MDC(LogKeys.URI, classUri)}") new ExecutorClassLoader(conf, env, classUri, parent, userClassPathFirst) } else { parent } - logInfo(log"Created or updated repl class loader ${LogMDC(CLASS_LOADER, classLoader)}" + - log" for ${LogMDC(SESSION_ID, sessionUUID)}.") + logInfo(log"Created or updated repl class loader ${MDC(CLASS_LOADER, classLoader)}" + + log" for ${MDC(SESSION_ID, sessionUUID)}.") classLoader } @@ -1191,16 +1191,16 @@ private[spark] class Executor( // Fetch missing dependencies for ((name, timestamp) <- newFiles if state.currentFiles.getOrElse(name, -1L) < timestamp) { - logInfo(log"Fetching ${LogMDC(FILE_NAME, name)} with" + - log" timestamp ${LogMDC(TIMESTAMP, timestamp)}") + logInfo(log"Fetching ${MDC(FILE_NAME, name)} with" + + log" timestamp ${MDC(TIMESTAMP, timestamp)}") // Fetch file with useCache mode, close cache for local mode. Utils.fetchFile(name, root, conf, hadoopConf, timestamp, useCache = !isLocal) state.currentFiles(name) = timestamp } for ((name, timestamp) <- newArchives if state.currentArchives.getOrElse(name, -1L) < timestamp) { - logInfo(log"Fetching ${LogMDC(ARCHIVE_NAME, name)} with" + - log" timestamp ${LogMDC(TIMESTAMP, timestamp)}") + logInfo(log"Fetching ${MDC(ARCHIVE_NAME, name)} with" + + log" timestamp ${MDC(TIMESTAMP, timestamp)}") val sourceURI = new URI(name) val uriToDownload = Utils.getUriBuilder(sourceURI).fragment(null).build() val source = Utils.fetchFile(uriToDownload.toString, Utils.createTempDir(), conf, @@ -1209,10 +1209,10 @@ private[spark] class Executor( root, if (sourceURI.getFragment != null) sourceURI.getFragment else source.getName) logInfo( - log"Unpacking an archive ${LogMDC(ARCHIVE_NAME, name)}" + - log" (${LogMDC(BYTE_SIZE, source.length)} bytes)" + - log" from ${LogMDC(SOURCE_PATH, source.getAbsolutePath)}" + - log" to ${LogMDC(DESTINATION_PATH, dest.getAbsolutePath)}") + log"Unpacking an archive ${MDC(ARCHIVE_NAME, name)}" + + log" (${MDC(BYTE_SIZE, source.length)} bytes)" + + log" from ${MDC(SOURCE_PATH, source.getAbsolutePath)}" + + log" to ${MDC(DESTINATION_PATH, dest.getAbsolutePath)}") Utils.deleteRecursively(dest) Utils.unpack(source, dest) state.currentArchives(name) = timestamp @@ -1223,8 +1223,8 @@ private[spark] class Executor( .orElse(state.currentJars.get(localName)) .getOrElse(-1L) if (currentTimeStamp < timestamp) { - logInfo(log"Fetching ${LogMDC(JAR_URL, name)} with" + - log" timestamp ${LogMDC(TIMESTAMP, timestamp)}") + logInfo(log"Fetching ${MDC(JAR_URL, name)} with" + + log" timestamp ${MDC(TIMESTAMP, timestamp)}") // Fetch file with useCache mode, close cache for local mode. Utils.fetchFile(name, root, conf, hadoopConf, timestamp, useCache = !isLocal) @@ -1232,8 +1232,8 @@ private[spark] class Executor( // Add it to our class loader val url = new File(root, localName).toURI.toURL if (!state.urlClassLoader.getURLs().contains(url)) { - logInfo(log"Adding ${LogMDC(LogKeys.URL, url)} to" + - log" class loader ${LogMDC(UUID, state.sessionUUID)}") + logInfo(log"Adding ${MDC(LogKeys.URL, url)} to" + + log" class loader ${MDC(UUID, state.sessionUUID)}") state.urlClassLoader.addURL(url) if (isStubbingEnabledForState(state.sessionUUID)) { renewClassLoader = true @@ -1298,7 +1298,7 @@ private[spark] class Executor( heartbeatFailures += 1 if (heartbeatFailures >= HEARTBEAT_MAX_FAILURES) { logError(log"Exit as unable to send heartbeats to driver " + - log"more than ${LogMDC(MAX_ATTEMPTS, HEARTBEAT_MAX_FAILURES)} times") + log"more than ${MDC(MAX_ATTEMPTS, HEARTBEAT_MAX_FAILURES)} times") System.exit(ExecutorExitCode.HEARTBEAT_FAILURE) } } @@ -1309,7 +1309,7 @@ private[spark] class Executor( if (runner != null) { runner.theadDump() } else { - logWarning(log"Failed to dump thread for task ${LogMDC(TASK_ID, taskId)}") + logWarning(log"Failed to dump thread for task ${MDC(TASK_ID, taskId)}") None } } @@ -1328,8 +1328,8 @@ private[spark] object Executor extends Logging { try { // This tests if any class initialization error is thrown val testKey = System.nanoTime().toString - MDC.put(testKey, "testValue") - MDC.remove(testKey) + SLF4JMDC.put(testKey, "testValue") + SLF4JMDC.remove(testKey) true } catch { diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorClassLoader.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorClassLoader.scala index 6f8138da6f4fb..faea70933071a 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorClassLoader.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorClassLoader.scala @@ -30,7 +30,7 @@ import org.apache.xbean.asm9.Opcodes._ import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.util.ParentClassLoader /** diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorLogUrlHandler.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorLogUrlHandler.scala index 2202489509fc4..9807cb2c81e45 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorLogUrlHandler.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorLogUrlHandler.scala @@ -21,7 +21,7 @@ import java.util.concurrent.atomic.AtomicBoolean import scala.util.matching.Regex -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys private[spark] class ExecutorLogUrlHandler(logUrlPattern: Option[String]) extends Logging { diff --git a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapRedCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapRedCommitProtocol.scala index 44f8d7cd63635..358424abefba5 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapRedCommitProtocol.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapRedCommitProtocol.scala @@ -21,7 +21,6 @@ import org.apache.hadoop.mapred._ import org.apache.hadoop.mapreduce.{TaskAttemptContext => NewTaskAttemptContext} import org.apache.spark.internal.LogKeys -import org.apache.spark.internal.MDC /** * An [[FileCommitProtocol]] implementation backed by an underlying Hadoop OutputCommitter diff --git a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala index 79218dffff9e3..d7bda5bbe721a 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala @@ -29,7 +29,7 @@ import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.mapred.SparkHadoopMapRedUtil diff --git a/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala index db961b3c42f4c..de136a86a2cc3 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala @@ -32,7 +32,7 @@ import org.apache.hadoop.mapreduce.task.{TaskAttemptContextImpl => NewTaskAttemp import org.apache.spark.{SerializableWritable, SparkConf, SparkException, TaskContext} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{DURATION, JOB_ID, TASK_ATTEMPT_ID} import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage import org.apache.spark.rdd.{HadoopRDD, RDD} diff --git a/core/src/main/scala/org/apache/spark/internal/plugin/PluginContainer.scala b/core/src/main/scala/org/apache/spark/internal/plugin/PluginContainer.scala index a0c07bd75f885..51b29e1a0ec68 100644 --- a/core/src/main/scala/org/apache/spark/internal/plugin/PluginContainer.scala +++ b/core/src/main/scala/org/apache/spark/internal/plugin/PluginContainer.scala @@ -22,7 +22,7 @@ import scala.util.{Either, Left, Right} import org.apache.spark.{SparkContext, SparkEnv, TaskFailedReason} import org.apache.spark.api.plugin._ -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.config._ import org.apache.spark.resource.ResourceInformation import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/internal/plugin/PluginEndpoint.scala b/core/src/main/scala/org/apache/spark/internal/plugin/PluginEndpoint.scala index 6ff918979c9ed..2c2f2f022a7a3 100644 --- a/core/src/main/scala/org/apache/spark/internal/plugin/PluginEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/internal/plugin/PluginEndpoint.scala @@ -18,7 +18,7 @@ package org.apache.spark.internal.plugin import org.apache.spark.api.plugin.DriverPlugin -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.rpc.{IsolatedThreadSafeRpcEndpoint, RpcCallContext, RpcEnv} diff --git a/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala b/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala index 0aaa222e6195e..e9745a7c2ba67 100644 --- a/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala +++ b/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala @@ -24,7 +24,7 @@ import org.apache.hadoop.mapreduce.{OutputCommitter => MapReduceOutputCommitter} import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.executor.CommitDeniedException -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{TASK_ATTEMPT_ID, TOTAL_TIME} import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala b/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala index 7098961d1649a..8b31d02cf97e3 100644 --- a/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala +++ b/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala @@ -21,7 +21,7 @@ import javax.annotation.concurrent.GuardedBy import scala.collection.mutable -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ /** diff --git a/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala index 24fcb5b17f388..7cde44b553321 100644 --- a/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala +++ b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala @@ -20,7 +20,7 @@ package org.apache.spark.memory import javax.annotation.concurrent.GuardedBy import org.apache.spark.SparkException -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.storage.BlockId import org.apache.spark.storage.memory.MemoryStore diff --git a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala index 212b54239ee60..db51f14415e1a 100644 --- a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala @@ -24,7 +24,7 @@ import scala.jdk.CollectionConverters._ import scala.util.control.NonFatal import org.apache.spark.{SparkConf, SparkIllegalArgumentException} -import org.apache.spark.internal.{config, Logging, LogKeys, MDC} +import org.apache.spark.internal.{config, Logging, LogKeys} import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config.Tests._ import org.apache.spark.internal.config.UNMANAGED_MEMORY_POLLING_INTERVAL diff --git a/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala b/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala index 965468ac2418f..b192a9f9babd1 100644 --- a/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala +++ b/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala @@ -24,7 +24,7 @@ import scala.jdk.CollectionConverters._ import org.apache.spark.SparkEnv import org.apache.spark.executor.ProcfsMetricsGetter -import org.apache.spark.internal.{config, Logging, MDC} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.LogKeys._ import org.apache.spark.memory.MemoryManager diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala index a845feeb67ff0..b975cefb336c2 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala @@ -25,7 +25,7 @@ import scala.jdk.CollectionConverters._ import scala.util.matching.Regex import org.apache.spark.SparkConf -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.PATH import org.apache.spark.internal.config.METRICS_CONF import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 709ce0060e150..af1f54007248d 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -26,7 +26,7 @@ import com.codahale.metrics.{Metric, MetricRegistry} import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.{SecurityManager, SparkConf} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys import org.apache.spark.internal.config._ import org.apache.spark.metrics.sink.{MetricsServlet, PrometheusServlet, Sink} diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/StatsdSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/StatsdSink.scala index 30b10d64882ac..4b74354aa15e3 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/StatsdSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/StatsdSink.scala @@ -22,7 +22,7 @@ import java.util.concurrent.TimeUnit import com.codahale.metrics.{Metric, MetricFilter, MetricRegistry} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.PREFIX import org.apache.spark.metrics.MetricsSystem diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala index a922eb336c28f..03810292bd05c 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala @@ -23,7 +23,7 @@ import scala.jdk.CollectionConverters._ import scala.reflect.ClassTag import org.apache.spark.SparkException -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.network.BlockDataManager import org.apache.spark.network.buffer.NioManagedBuffer diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index 7ceb50db5966a..fe55518d80004 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -30,7 +30,7 @@ import com.codahale.metrics.{Metric, MetricSet} import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.ExecutorDeadException -import org.apache.spark.internal.{config, LogKeys, MDC} +import org.apache.spark.internal.{config, Logging, LogKeys} import org.apache.spark.network._ import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.client.{RpcResponseCallback, TransportClientBootstrap} @@ -57,7 +57,7 @@ private[spark] class NettyBlockTransferService( _port: Int, numCores: Int, driverEndPointRef: RpcEndpointRef = null) - extends BlockTransferService { + extends BlockTransferService with Logging { // TODO: Don't use Java serialization, use a more cross-version compatible serialization format. private val serializer = serializerManager.getSerializer(scala.reflect.classTag[Any], false) @@ -196,9 +196,9 @@ private[spark] class NettyBlockTransferService( override def onFailure(e: Throwable): Unit = { if (asStream) { - logger.error(s"Error while uploading {} as stream", e, MDC.of(LogKeys.BLOCK_ID, blockId)) + logger.error(s"Error while uploading {} as stream", e, MDC(LogKeys.BLOCK_ID, blockId)) } else { - logger.error(s"Error while uploading {}", e, MDC.of(LogKeys.BLOCK_ID, blockId)) + logger.error(s"Error while uploading {}", e, MDC(LogKeys.BLOCK_ID, blockId)) } result.failure(e) } diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 44ce1bfa8c5c8..a16bd98a74669 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -39,7 +39,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.errors.SparkCoreErrors -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config._ import org.apache.spark.rdd.HadoopRDD.HadoopMapPartitionsWithSplitRDD diff --git a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala index 0edd3912afd6b..e71cf4d002a74 100644 --- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala @@ -25,7 +25,7 @@ import org.apache.spark.{Partition, SparkContext, TaskContext} import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.api.java.function.{Function => JFunction} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.util.NextIterator diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 27e4948ce1297..cdf90a03a554a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -39,7 +39,7 @@ import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.errors.SparkCoreErrors -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config._ import org.apache.spark.rdd.NewHadoopRDD.NewHadoopMapPartitionsWithSplitRDD diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index c0966dd5ede14..956620e0b2c23 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -35,7 +35,7 @@ import org.apache.hadoop.mapreduce.{Job => NewAPIHadoopJob, OutputFormat => NewO import org.apache.spark._ import org.apache.spark.Partitioner.defaultPartitioner import org.apache.spark.errors.SparkCoreErrors -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config.SPECULATION_ENABLED import org.apache.spark.internal.io._ diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala index b2b6d2a2959da..5a9e51026ba98 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -35,7 +35,6 @@ import scala.reflect.ClassTag import org.apache.spark.{Partition, TaskContext} import org.apache.spark.errors.SparkCoreErrors import org.apache.spark.internal.LogKeys.{COMMAND, ERROR, PATH} -import org.apache.spark.internal.MDC import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index b3d412ed04949..117b2925710d3 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -36,7 +36,7 @@ import org.apache.spark.Partitioner._ import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} import org.apache.spark.api.java.JavaRDD import org.apache.spark.errors.SparkCoreErrors -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config._ import org.apache.spark.internal.config.RDD_LIMIT_SCALE_UP_FACTOR diff --git a/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala index ee13acc650337..fd42cea795d60 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala @@ -29,7 +29,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.errors.SparkCoreErrors -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config.{BUFFER_SIZE, CACHE_CHECKPOINT_PREFERRED_LOCS_EXPIRE_TIME, CHECKPOINT_COMPRESS} import org.apache.spark.io.CompressionCodec diff --git a/core/src/main/scala/org/apache/spark/rdd/ReliableRDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/ReliableRDDCheckpointData.scala index b468a38fcf229..9f80d53d2c0bf 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ReliableRDDCheckpointData.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ReliableRDDCheckpointData.scala @@ -23,7 +23,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark._ import org.apache.spark.errors.SparkCoreErrors -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{NEW_RDD_ID, RDD_CHECKPOINT_DIR, RDD_ID} import org.apache.spark.internal.config.CLEANER_REFERENCE_TRACKING_CLEAN_CHECKPOINTS diff --git a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala index 118660ef69476..14e8bade2200a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala @@ -23,7 +23,7 @@ import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapred.SequenceFileOutputFormat -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} /** * Extra functions available on RDDs of (key, value) pairs to create a Hadoop SequenceFile, diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceDiscoveryScriptPlugin.scala b/core/src/main/scala/org/apache/spark/resource/ResourceDiscoveryScriptPlugin.scala index 51de7e2b9ac70..b11c88828e065 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceDiscoveryScriptPlugin.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceDiscoveryScriptPlugin.scala @@ -23,7 +23,7 @@ import java.util.Optional import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.resource.ResourceDiscoveryPlugin -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys import org.apache.spark.util.Utils.executeAndGetOutput diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala index 0baa578764d08..89b79577c3e58 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala @@ -26,7 +26,7 @@ import scala.jdk.CollectionConverters._ import org.apache.spark.{SparkConf, SparkContext, SparkEnv, SparkException} import org.apache.spark.annotation.{Evolving, Since} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Python.PYSPARK_EXECUTOR_MEMORY diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala index 6a6b5067f70f2..10121f6ef2667 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala @@ -23,7 +23,7 @@ import scala.collection.mutable.HashMap import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.annotation.Evolving -import org.apache.spark.internal.{config, Logging, MDC} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.LogKeys import org.apache.spark.internal.config.Tests._ import org.apache.spark.scheduler.{LiveListenerBus, SparkListenerResourceProfileAdded} diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala b/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala index 78c45cdc75418..8e11923a9e8f0 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala @@ -28,7 +28,7 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.resource.ResourceDiscoveryPlugin -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config.{EXECUTOR_CORES, RESOURCES_DISCOVERY_PLUGIN, SPARK_TASK_PREFIX} import org.apache.spark.internal.config.Tests.RESOURCES_WARNING_TESTING diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala b/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala index 8acfef38659c0..faa263c7db6a7 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala @@ -25,7 +25,7 @@ import scala.jdk.CollectionConverters._ import scala.util.control.NonFatal import org.apache.spark.{SparkEnv, SparkException} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.network.client.RpcResponseCallback import org.apache.spark.rpc._ diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala b/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala index 0de67a65593b1..dfaa9ea436458 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala @@ -22,7 +22,7 @@ import javax.annotation.concurrent.GuardedBy import scala.util.control.NonFatal import org.apache.spark.SparkException -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.rpc.{RpcAddress, RpcEndpoint, ThreadSafeRpcEndpoint} diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/MessageLoop.scala b/core/src/main/scala/org/apache/spark/rpc/netty/MessageLoop.scala index 2fd1c6d7fe71e..cce455270df43 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/MessageLoop.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/MessageLoop.scala @@ -22,7 +22,7 @@ import java.util.concurrent._ import scala.util.control.NonFatal import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.ERROR import org.apache.spark.internal.config.EXECUTOR_ID import org.apache.spark.internal.config.Network._ diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala index c2688610fe8b1..a4f6d5438bf3d 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala @@ -30,7 +30,7 @@ import scala.util.{DynamicVariable, Failure, Success, Try} import scala.util.control.NonFatal import org.apache.spark.{SecurityManager, SparkConf, SparkContext} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config.EXECUTOR_ID import org.apache.spark.internal.config.Network._ diff --git a/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala b/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala index 16e9211b54851..3931b8ff5190e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala @@ -23,7 +23,7 @@ import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong} import com.codahale.metrics.{Gauge, Timer} import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config._ import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index baf0ed4df5309..30eb49b0c0798 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -37,7 +37,7 @@ import org.apache.spark._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.errors.SparkCoreErrors import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} -import org.apache.spark.internal.{config, Logging, LogKeys, MDC} +import org.apache.spark.internal.{config, Logging, LogKeys} import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config.{LEGACY_ABORT_STAGE_AFTER_KILL_TASKS, RDD_CACHE_VISIBILITY_TRACKING_ENABLED} import org.apache.spark.internal.config.Tests.TEST_NO_STAGE_RETRY diff --git a/core/src/main/scala/org/apache/spark/scheduler/HealthTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/HealthTracker.scala index 82ec0ef91f4fc..fe62e39432250 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/HealthTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/HealthTracker.scala @@ -22,7 +22,7 @@ import java.util.concurrent.atomic.AtomicReference import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import org.apache.spark.{ExecutorAllocationClient, SparkConf, SparkContext} -import org.apache.spark.internal.{config, Logging, MDC} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.LogKeys._ import org.apache.spark.util.{Clock, SystemClock, Utils} diff --git a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala index 7251eb2c86ea1..656f9875de2e2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala @@ -30,7 +30,7 @@ import com.codahale.metrics.{Counter, MetricRegistry, Timer} import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.errors.SparkCoreErrors -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CLASS_NAME, MAX_SIZE} import org.apache.spark.internal.config._ import org.apache.spark.metrics.MetricsSystem diff --git a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala index a769c3fa14b62..3e6f0fe3bfc44 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala @@ -20,7 +20,7 @@ package org.apache.spark.scheduler import scala.collection.mutable import org.apache.spark._ -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.rpc.{RpcCallContext, RpcEndpoint, RpcEndpointRef, RpcEnv} import org.apache.spark.util.{RpcUtils, ThreadUtils} diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index 0aa74190d9f3d..9b026207988c2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -24,7 +24,7 @@ import scala.io.{Codec, Source} import com.fasterxml.jackson.core.JsonParseException import com.fasterxml.jackson.databind.exc.UnrecognizedPropertyException -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.scheduler.ReplayListenerBus._ import org.apache.spark.util.JsonProtocol diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala index bea49fb279ee3..06b6045cccd99 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala @@ -26,7 +26,7 @@ import scala.xml.{Node, XML} import org.apache.hadoop.fs.Path import org.apache.spark.SparkContext -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config.{SCHEDULER_ALLOCATION_FILE, SCHEDULER_MODE} diff --git a/core/src/main/scala/org/apache/spark/scheduler/StatsReportListener.scala b/core/src/main/scala/org/apache/spark/scheduler/StatsReportListener.scala index e46dde5561a26..c8ffaefba3d64 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StatsReportListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StatsReportListener.scala @@ -21,7 +21,7 @@ import scala.collection.mutable import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.util.{Distribution, Utils} diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index e21ec77ce69ec..0e5e2c9faa05a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -22,7 +22,7 @@ import java.util.Properties import org.apache.spark._ import org.apache.spark.executor.TaskMetrics -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.config.APP_CALLER_CONTEXT import org.apache.spark.internal.plugin.PluginContainer import org.apache.spark.memory.{MemoryMode, TaskMemoryManager} diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index 97c539bb05a58..70f58747506cc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -25,7 +25,7 @@ import scala.util.control.NonFatal import org.apache.spark._ import org.apache.spark.TaskState.TaskState -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.CLASS_LOADER import org.apache.spark.serializer.{SerializerHelper, SerializerInstance} import org.apache.spark.util.{LongAccumulator, ThreadUtils, Utils} diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 13018da5bc274..1351d8c778b58 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -33,7 +33,7 @@ import org.apache.spark.InternalAccumulator.{input, shuffleRead} import org.apache.spark.TaskState.TaskState import org.apache.spark.errors.SparkCoreErrors import org.apache.spark.executor.ExecutorMetrics -import org.apache.spark.internal.{config, Logging, LogKeys, MDC} +import org.apache.spark.internal.{config, Logging, LogKeys} import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config._ import org.apache.spark.resource.ResourceProfile diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetExcludeList.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetExcludeList.scala index 3637305293107..c1b09567013f0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetExcludeList.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetExcludeList.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler import scala.collection.mutable.{HashMap, HashSet} import org.apache.spark.SparkConf -import org.apache.spark.internal.{config, Logging, LogKeys, MDC} +import org.apache.spark.internal.{config, Logging, LogKeys} import org.apache.spark.util.Clock /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 0eaf138d3eb8d..69e0a10a34b28 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -30,7 +30,7 @@ import org.apache.spark.InternalAccumulator import org.apache.spark.InternalAccumulator.{input, shuffleRead} import org.apache.spark.TaskState.TaskState import org.apache.spark.errors.SparkCoreErrors -import org.apache.spark.internal.{config, Logging, LogKeys, MDC} +import org.apache.spark.internal.{config, Logging, LogKeys} import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config._ import org.apache.spark.scheduler.SchedulingMode._ diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 4b3a16b4d3f60..49590f0940f76 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -32,7 +32,7 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.errors.SparkCoreErrors import org.apache.spark.executor.ExecutorLogUrlHandler -import org.apache.spark.internal.{config, Logging, MDC} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.LogKeys import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config._ diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index eb408a95589f7..061b54914c839 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -27,7 +27,7 @@ import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.deploy.{ApplicationDescription, Command} import org.apache.spark.deploy.client.{StandaloneAppClient, StandaloneAppClientListener} import org.apache.spark.executor.ExecutorExitCode -import org.apache.spark.internal.{config, Logging, LogKeys, MDC} +import org.apache.spark.internal.{config, Logging, LogKeys} import org.apache.spark.internal.config.EXECUTOR_REMOVE_DELAY import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} diff --git a/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala b/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala index 57505c87f879e..a98672fd7db4a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala @@ -25,7 +25,7 @@ import scala.jdk.CollectionConverters._ import org.apache.spark._ import org.apache.spark.errors.SparkCoreErrors -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.config._ import org.apache.spark.resource.ResourceProfile.UNKNOWN_RESOURCE_PROFILE_ID import org.apache.spark.scheduler._ diff --git a/core/src/main/scala/org/apache/spark/security/CryptoStreamUtils.scala b/core/src/main/scala/org/apache/spark/security/CryptoStreamUtils.scala index 1ee46d51ce70b..8bdb80d65207d 100644 --- a/core/src/main/scala/org/apache/spark/security/CryptoStreamUtils.scala +++ b/core/src/main/scala/org/apache/spark/security/CryptoStreamUtils.scala @@ -31,7 +31,7 @@ import org.apache.commons.crypto.random._ import org.apache.commons.crypto.stream._ import org.apache.spark.SparkConf -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config._ import org.apache.spark.network.util.{CryptoUtils, JavaUtils} diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 000ba8d79bc02..28261a4520a28 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -40,7 +40,7 @@ import org.roaringbitmap.RoaringBitmap import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.python.PythonBroadcast -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.CLASS_NAME import org.apache.spark.internal.config.Kryo._ import org.apache.spark.internal.io.FileCommitProtocol._ diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index bf3117a9a9b12..9b27f648134ee 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -29,7 +29,7 @@ import com.google.common.cache.CacheBuilder import org.apache.spark.{SecurityManager, SparkConf, SparkEnv, SparkException} import org.apache.spark.errors.SparkCoreErrors -import org.apache.spark.internal.{config, Logging, LogKeys, MDC} +import org.apache.spark.internal.{config, Logging, LogKeys} import org.apache.spark.internal.LogKeys._ import org.apache.spark.io.NioBufferedFileInputStream import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala index 4e3191e44fbdf..548ecb399d5ff 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala @@ -27,7 +27,7 @@ import scala.util.control.NonFatal import org.apache.spark.{SecurityManager, ShuffleDependency, SparkConf, SparkContext, SparkEnv} import org.apache.spark.annotation.Since import org.apache.spark.executor.{CoarseGrainedExecutorBackend, ExecutorBackend} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config._ import org.apache.spark.launcher.SparkLauncher diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala index be42af092f24a..47d54ae4f10b0 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala @@ -18,7 +18,7 @@ package org.apache.spark.shuffle import org.apache.spark.{ShuffleDependency, SparkEnv, TaskContext} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{NUM_MERGER_LOCATIONS, SHUFFLE_ID, STAGE_ID} import org.apache.spark.scheduler.MapStatus diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala index 5c93bf4bf77a0..52856427cb37a 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -26,7 +26,7 @@ import scala.jdk.CollectionConverters._ import org.apache.spark._ import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.config.CPUS_PER_TASK import org.apache.spark.internal.config.Status._ import org.apache.spark.resource.ResourceProfile.CPUS diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala index 87f876467c30e..a7f3fde9e6f6f 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala @@ -25,7 +25,7 @@ import scala.collection.mutable.HashMap import scala.jdk.CollectionConverters._ import org.apache.spark.{JobExecutionStatus, SparkConf, SparkContext} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.PATH import org.apache.spark.internal.config.Status.LIVE_UI_LOCAL_STORE_DIR import org.apache.spark.status.AppStatusUtils.getQuantilesValue diff --git a/core/src/main/scala/org/apache/spark/status/KVUtils.scala b/core/src/main/scala/org/apache/spark/status/KVUtils.scala index 49b77111abafd..76fb654f8da2d 100644 --- a/core/src/main/scala/org/apache/spark/status/KVUtils.scala +++ b/core/src/main/scala/org/apache/spark/status/KVUtils.scala @@ -31,7 +31,7 @@ import org.rocksdb.RocksDBException import org.apache.spark.SparkConf import org.apache.spark.deploy.history.{FsHistoryProvider, FsHistoryProviderMetadata} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config.History import org.apache.spark.internal.config.History.HYBRID_STORE_DISK_BACKEND diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 96a96756d7709..1e1cb8bf9fd53 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -40,7 +40,7 @@ import com.google.common.cache.CacheBuilder import org.apache.spark._ import org.apache.spark.errors.SparkCoreErrors import org.apache.spark.executor.{DataReadMethod, ExecutorExitCode} -import org.apache.spark.internal.{config, Logging, MDC} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config.{Network, RDD_CACHE_VISIBILITY_TRACKING_ENABLED, Tests} import org.apache.spark.memory.{MemoryManager, MemoryMode} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala index 19807453ee28c..1a067fab04c1d 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala @@ -26,7 +26,7 @@ import scala.util.control.NonFatal import org.apache.spark._ import org.apache.spark.errors.SparkCoreErrors -import org.apache.spark.internal.{config, Logging, MDC} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.LogKeys._ import org.apache.spark.shuffle.ShuffleBlockInfo import org.apache.spark.storage.BlockManagerMessages.ReplicateBlock diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index 276bd63e14237..16c2cbbe7fdc4 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -23,7 +23,7 @@ import scala.concurrent.Future import org.apache.spark.SparkConf import org.apache.spark.errors.SparkCoreErrors -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.storage.BlockManagerMessages._ diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index 858db498e83ad..f63d4a55669aa 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -31,7 +31,7 @@ import com.google.common.cache.CacheBuilder import org.apache.spark.{MapOutputTrackerMaster, SparkConf, SparkContext, SparkEnv} import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.internal.{config, Logging, MDC} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config.RDD_CACHE_VISIBILITY_TRACKING_ENABLED import org.apache.spark.network.shuffle.{ExternalBlockStoreClient, RemoteBlockPushResolver} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerStorageEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerStorageEndpoint.scala index f29e8778da037..54329c5b1e514 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerStorageEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerStorageEndpoint.scala @@ -20,7 +20,7 @@ package org.apache.spark.storage import scala.concurrent.{ExecutionContext, ExecutionContextExecutorService, Future} import org.apache.spark.{MapOutputTracker, SparkEnv} -import org.apache.spark.internal.{Logging, MDC, MessageWithContext} +import org.apache.spark.internal.{Logging, MessageWithContext} import org.apache.spark.internal.LogKeys.{BLOCK_ID, BROADCAST_ID, RDD_ID, SHUFFLE_ID} import org.apache.spark.rpc.{IsolatedThreadSafeRpcEndpoint, RpcCallContext, RpcEnv} import org.apache.spark.storage.BlockManagerMessages._ diff --git a/core/src/main/scala/org/apache/spark/storage/BlockReplicationPolicy.scala b/core/src/main/scala/org/apache/spark/storage/BlockReplicationPolicy.scala index 5186cbfa217cc..9126dc8a83561 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockReplicationPolicy.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockReplicationPolicy.scala @@ -21,7 +21,7 @@ import scala.collection.mutable import scala.util.Random import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ /** diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 94a0ea1ecaef0..df29ddf6999f5 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -30,7 +30,7 @@ import com.fasterxml.jackson.module.scala.DefaultScalaModule import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.errors.SparkCoreErrors import org.apache.spark.executor.ExecutorExitCode -import org.apache.spark.internal.{config, Logging, MDC} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.LogKeys.{MERGE_DIR_NAME, PATH} import org.apache.spark.network.shuffle.ExecutorDiskUtils import org.apache.spark.storage.DiskBlockManager.ATTEMPT_ID_KEY diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala index efcdb7fa8c69e..9964e64724f6b 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala @@ -24,7 +24,7 @@ import java.util.zip.Checksum import org.apache.spark.SparkException import org.apache.spark.errors.SparkCoreErrors -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.io.MutableCheckedOutputStream import org.apache.spark.serializer.{SerializationStream, SerializerInstance, SerializerManager} diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index d17923b172a89..b304d7bc32a84 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -29,7 +29,7 @@ import com.google.common.io.Closeables import io.netty.channel.DefaultFileRegion import org.apache.spark.{SecurityManager, SparkConf, SparkException} -import org.apache.spark.internal.{config, Logging, MDC} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.LogKeys._ import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.network.util.{AbstractFileRegion, JavaUtils} diff --git a/core/src/main/scala/org/apache/spark/storage/FallbackStorage.scala b/core/src/main/scala/org/apache/spark/storage/FallbackStorage.scala index 0f2bfaede4454..c086be0bfe806 100644 --- a/core/src/main/scala/org/apache/spark/storage/FallbackStorage.scala +++ b/core/src/main/scala/org/apache/spark/storage/FallbackStorage.scala @@ -28,7 +28,7 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config.{STORAGE_DECOMMISSION_FALLBACK_STORAGE_CLEANUP, STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH} import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} diff --git a/core/src/main/scala/org/apache/spark/storage/PushBasedFetchHelper.scala b/core/src/main/scala/org/apache/spark/storage/PushBasedFetchHelper.scala index 8a3ca3066961c..16a68bbf50e25 100644 --- a/core/src/main/scala/org/apache/spark/storage/PushBasedFetchHelper.scala +++ b/core/src/main/scala/org/apache/spark/storage/PushBasedFetchHelper.scala @@ -28,7 +28,7 @@ import org.roaringbitmap.RoaringBitmap import org.apache.spark.MapOutputTracker import org.apache.spark.MapOutputTracker.SHUFFLE_PUSH_MAP_ID -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.network.shuffle.{BlockStoreClient, MergedBlockMeta, MergedBlocksMetaListener} import org.apache.spark.shuffle.ShuffleReadMetricsReporter diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index 533462246b625..b2f185bc590fd 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -35,7 +35,7 @@ import org.roaringbitmap.RoaringBitmap import org.apache.spark.{MapOutputTracker, SparkException, TaskContext} import org.apache.spark.MapOutputTracker.SHUFFLE_PUSH_MAP_ID import org.apache.spark.errors.SparkCoreErrors -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.network.shuffle._ diff --git a/core/src/main/scala/org/apache/spark/storage/TopologyMapper.scala b/core/src/main/scala/org/apache/spark/storage/TopologyMapper.scala index f1dca53c7e3b1..45d3a4acb9ca9 100644 --- a/core/src/main/scala/org/apache/spark/storage/TopologyMapper.scala +++ b/core/src/main/scala/org/apache/spark/storage/TopologyMapper.scala @@ -19,7 +19,7 @@ package org.apache.spark.storage import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.internal.{config, Logging, MDC} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.LogKeys._ import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala index 6746bbd490c42..9bb38fc43938d 100644 --- a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala @@ -30,7 +30,7 @@ import scala.util.control.NonFatal import com.google.common.io.ByteStreams import org.apache.spark.{SparkConf, SparkException, TaskContext} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config.{STORAGE_UNROLL_MEMORY_THRESHOLD, UNROLL_MEMORY_CHECK_PERIOD, UNROLL_MEMORY_GROWTH_FACTOR} import org.apache.spark.memory.{MemoryManager, MemoryMode} diff --git a/core/src/main/scala/org/apache/spark/ui/DriverLogPage.scala b/core/src/main/scala/org/apache/spark/ui/DriverLogPage.scala index 8b4eebc26b3ba..dca85b53178ad 100644 --- a/core/src/main/scala/org/apache/spark/ui/DriverLogPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/DriverLogPage.scala @@ -21,7 +21,7 @@ import scala.xml.{Node, Unparsed} import jakarta.servlet.http.HttpServletRequest import org.apache.spark.SparkConf -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{LOG_TYPE, PATH} import org.apache.spark.internal.config.DRIVER_LOG_LOCAL_DIR import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 068f9b0b2cbcb..bd7848c764375 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -40,7 +40,7 @@ import org.json4s.JValue import org.json4s.jackson.JsonMethods.{pretty, render} import org.apache.spark.{SecurityManager, SparkConf, SSLOptions} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config.UI._ diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index c6a0b6ae27ceb..a23764f98d49d 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -23,7 +23,7 @@ import jakarta.servlet.http.{HttpServlet, HttpServletRequest, HttpServletRespons import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.{SecurityManager, SparkConf, SparkContext} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CLASS_NAME, WEB_URL} import org.apache.spark.internal.config.DRIVER_LOG_LOCAL_DIR import org.apache.spark.internal.config.UI._ diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index 60d4e5db99d7e..c0d0fb50ae36b 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -29,7 +29,7 @@ import org.eclipse.jetty.servlet.{FilterHolder, FilterMapping, ServletContextHan import org.json4s.JsonAST.{JNothing, JValue} import org.apache.spark.{SecurityManager, SparkConf, SSLOptions} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config._ import org.apache.spark.ui.JettyUtils._ diff --git a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala index ee7f67233bbd5..cd057ed08c3c3 100644 --- a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala +++ b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala @@ -25,7 +25,7 @@ import scala.xml.Utility import org.apache.commons.text.StringEscapeUtils -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.rdd.DeterministicLevel import org.apache.spark.scheduler.StageInfo diff --git a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala index 1745498456213..7e2047526da03 100644 --- a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala +++ b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala @@ -23,7 +23,7 @@ import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicLong import org.apache.spark.{InternalAccumulator, SparkContext, TaskContext} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.util.AccumulatorContext.internOption diff --git a/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala b/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala index be5cd2ed2a7e1..63caa14215774 100644 --- a/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/DependencyUtils.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.deploy.SparkSubmit -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config._ diff --git a/core/src/main/scala/org/apache/spark/util/EventLoop.scala b/core/src/main/scala/org/apache/spark/util/EventLoop.scala index b9de661b63c4f..c15598a35651f 100644 --- a/core/src/main/scala/org/apache/spark/util/EventLoop.scala +++ b/core/src/main/scala/org/apache/spark/util/EventLoop.scala @@ -22,7 +22,7 @@ import java.util.concurrent.atomic.AtomicBoolean import scala.util.control.NonFatal -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.EVENT_LOOP /** diff --git a/core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala b/core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala index 5e50361b278aa..f84184f7c1faf 100644 --- a/core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala @@ -28,7 +28,7 @@ import org.apache.hadoop.fs.viewfs.ViewFileSystem import org.apache.hadoop.hdfs.DistributedFileSystem import org.apache.spark._ -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.metrics.source.HiveCatalogMetrics import org.apache.spark.util.ArrayImplicits._ diff --git a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala index 4f01cd6ac2136..0c0c932ac797c 100644 --- a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala @@ -26,7 +26,7 @@ import scala.util.control.NonFatal import com.codahale.metrics.Timer import org.apache.spark.SparkEnv -import org.apache.spark.internal.{config, Logging, MDC} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.LogKeys.{EVENT, LISTENER, TOTAL_TIME} import org.apache.spark.scheduler.EventLoggingListener import org.apache.spark.scheduler.SparkListenerEnvironmentUpdate diff --git a/core/src/main/scala/org/apache/spark/util/PeriodicCheckpointer.scala b/core/src/main/scala/org/apache/spark/util/PeriodicCheckpointer.scala index 7a98c4830db92..5c9c3d3564d00 100644 --- a/core/src/main/scala/org/apache/spark/util/PeriodicCheckpointer.scala +++ b/core/src/main/scala/org/apache/spark/util/PeriodicCheckpointer.scala @@ -23,7 +23,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.SparkContext -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ diff --git a/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala b/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala index ad9c3d3f10cfe..af93f781343d2 100644 --- a/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala +++ b/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala @@ -26,7 +26,7 @@ import scala.util.Try import org.apache.hadoop.fs.FileSystem import org.apache.spark.SparkConf -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.LogKeys.PATH import org.apache.spark.internal.config.SPARK_SHUTDOWN_TIMEOUT_MS diff --git a/core/src/main/scala/org/apache/spark/util/SignalUtils.scala b/core/src/main/scala/org/apache/spark/util/SignalUtils.scala index daa46fc4867da..e2f615b44115b 100644 --- a/core/src/main/scala/org/apache/spark/util/SignalUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/SignalUtils.scala @@ -24,7 +24,7 @@ import scala.jdk.CollectionConverters._ import org.slf4j.Logger import sun.misc.{Signal, SignalHandler} -import org.apache.spark.internal.{Logging, MDC, MessageWithContext} +import org.apache.spark.internal.{Logging, MessageWithContext} import org.apache.spark.internal.LogKeys._ /** diff --git a/core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala b/core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala index 25a6322743baa..a1aeee1d877c7 100644 --- a/core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala +++ b/core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala @@ -19,7 +19,7 @@ package org.apache.spark.util import org.apache.spark.SparkEnv import org.apache.spark.executor.{ExecutorExitCode, KilledByTaskReaperException} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.THREAD import org.apache.spark.internal.config.KILL_ON_FATAL_ERROR_DEPTH diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 9cf07793ef2af..affc7519eba86 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -67,7 +67,7 @@ import org.slf4j.Logger import org.apache.spark.{SPARK_VERSION, _} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.internal.{Logging, MDC, MessageWithContext} +import org.apache.spark.internal.{Logging, MessageWithContext} import org.apache.spark.internal.LogKeys import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config._ diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 16a2f4fb6cad9..d892fa0e47060 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -29,7 +29,7 @@ import com.google.common.io.ByteStreams import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.ShuffleWriteMetrics -import org.apache.spark.internal.{config, Logging, MDC} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.LogKeys._ import org.apache.spark.serializer.{DeserializationStream, Serializer, SerializerManager} import org.apache.spark.storage.{BlockId, BlockManager} diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 393cdbbef0a5a..8dd207b25bb94 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -28,7 +28,7 @@ import com.google.common.io.ByteStreams import org.apache.spark._ import org.apache.spark.executor.ShuffleWriteMetrics -import org.apache.spark.internal.{config, Logging, MDC} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.LogKeys.{NUM_BYTES, TASK_ATTEMPT_ID} import org.apache.spark.serializer._ import org.apache.spark.shuffle.{ShufflePartitionPairsWriter, ShuffleWriteMetricsReporter} diff --git a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala index 3ee4c9c0b401c..17713e83bb83e 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala @@ -18,7 +18,7 @@ package org.apache.spark.util.collection import org.apache.spark.SparkEnv -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.config._ import org.apache.spark.memory.{MemoryConsumer, MemoryMode, TaskMemoryManager} diff --git a/core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala b/core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala index 392d3ba7a980a..9854c99707b1c 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala @@ -32,7 +32,7 @@ import org.apache.logging.log4j.core.layout.PatternLayout import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config._ import org.apache.spark.network.util.JavaUtils diff --git a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala index 2fec1ff832c0a..39fde0cef534b 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala @@ -20,7 +20,7 @@ package org.apache.spark.util.logging import java.io.{File, FileOutputStream, InputStream, IOException} import org.apache.spark.SparkConf -import org.apache.spark.internal.{config, Logging, LogKeys, MDC} +import org.apache.spark.internal.{config, Logging, LogKeys} import org.apache.spark.internal.LogKeys._ import org.apache.spark.util.{IntParam, Utils} diff --git a/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala b/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala index 867014f235fe5..4967211ba97d3 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala @@ -23,8 +23,8 @@ import java.util.zip.GZIPOutputStream import com.google.common.io.Files import org.apache.spark.SparkConf -import org.apache.spark.internal.{config, MDC} import org.apache.spark.internal.LogKeys._ +import org.apache.spark.internal.config import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala b/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala index 310e895930943..7f4fc65fd92bf 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala @@ -20,7 +20,7 @@ package org.apache.spark.util.logging import java.text.SimpleDateFormat import java.util.{Calendar, Locale} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ /** diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala index 045149922c8ed..0c17e0b3bc4ac 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala @@ -21,7 +21,7 @@ import java.util.concurrent.TimeUnit import org.apache.spark.SparkContext import org.apache.spark.graphx.impl.{EdgePartitionBuilder, GraphImpl} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.TOTAL_TIME import org.apache.spark.storage.StorageLevel diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala index 823143f9b9abb..5b1e1ca6565b4 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala @@ -20,7 +20,7 @@ package org.apache.spark.graphx import scala.reflect.ClassTag import org.apache.spark.graphx.util.PeriodicGraphCheckpointer -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.NUM_ITERATIONS import org.apache.spark.rdd.RDD import org.apache.spark.rdd.util.PeriodicRDDCheckpointer diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala index 4fe010bfce785..2b3636ac1e333 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import breeze.linalg.{Vector => BV} import org.apache.spark.graphx._ -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.NUM_ITERATIONS import org.apache.spark.ml.linalg.{Vector, Vectors} diff --git a/hadoop-cloud/src/main/scala/org/apache/spark/internal/io/cloud/AbortableStreamBasedCheckpointFileManager.scala b/hadoop-cloud/src/main/scala/org/apache/spark/internal/io/cloud/AbortableStreamBasedCheckpointFileManager.scala index 599361009fcc5..4fe5bb007bf97 100644 --- a/hadoop-cloud/src/main/scala/org/apache/spark/internal/io/cloud/AbortableStreamBasedCheckpointFileManager.scala +++ b/hadoop-cloud/src/main/scala/org/apache/spark/internal/io/cloud/AbortableStreamBasedCheckpointFileManager.scala @@ -24,7 +24,7 @@ import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.sql.execution.streaming.AbstractFileContextBasedCheckpointFileManager import org.apache.spark.sql.execution.streaming.CheckpointFileManager.CancellableFSDataOutputStream diff --git a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala index 83b77510602b2..17d6be0ce7cd5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml import org.apache.spark.annotation.Since -import org.apache.spark.internal.{LogKeys, MDC} +import org.apache.spark.internal.{LogKeys} import org.apache.spark.ml.linalg.VectorUDT import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala index 7883a0dea54f1..d9238479e8031 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.Since -import org.apache.spark.internal.{LogKeys, MDC} +import org.apache.spark.internal.{LogKeys} import org.apache.spark.ml.{PredictionModel, Predictor, PredictorParams} import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param.ParamMap diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala index 0d163b761686d..89f1f7792c075 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala @@ -27,7 +27,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkException import org.apache.spark.annotation.Since -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{COUNT, RANGE} import org.apache.spark.ml.feature._ import org.apache.spark.ml.linalg._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 8c010f67f5e0a..d0824925780c0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -28,7 +28,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkException import org.apache.spark.annotation.Since -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.LogKeys.{COUNT, RANGE} import org.apache.spark.ml.feature._ import org.apache.spark.ml.impl.Utils diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala index 929bd1541ec6e..f62b951fd6505 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala @@ -29,7 +29,7 @@ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ import org.apache.spark.annotation.Since -import org.apache.spark.internal.{LogKeys, MDC} +import org.apache.spark.internal.{LogKeys} import org.apache.spark.ml._ import org.apache.spark.ml.attribute._ import org.apache.spark.ml.functions._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala index 61fab02cb4518..ea2c79d8a2181 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.Since -import org.apache.spark.internal.{LogKeys, MDC} +import org.apache.spark.internal.{LogKeys} import org.apache.spark.ml.linalg.{DenseVector, Vector, VectorUDT} import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.param.shared._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala index e7f930065486b..5d3e36be28082 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala @@ -23,7 +23,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.Since import org.apache.spark.broadcast.Broadcast -import org.apache.spark.internal.{LogKeys, MDC} +import org.apache.spark.internal.{LogKeys} import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.impl.Utils.{unpackUpperTriangular, EPSILON} import org.apache.spark.ml.linalg._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala index ccae39cedd20f..2abd82c712960 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala @@ -25,7 +25,6 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.Since import org.apache.spark.internal.LogKeys.{COST, INIT_MODE, NUM_ITERATIONS, TOTAL_TIME} -import org.apache.spark.internal.MDC import org.apache.spark.ml.{Estimator, Model, PipelineStage} import org.apache.spark.ml.feature.{Instance, InstanceBlock} import org.apache.spark.ml.linalg._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala index 8123438fd8878..52ed90415f1cd 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala @@ -21,7 +21,7 @@ import scala.collection.mutable.ArrayBuilder import org.apache.spark.{SparkException, SparkIllegalArgumentException} import org.apache.spark.annotation.Since -import org.apache.spark.internal.{LogKeys, MDC} +import org.apache.spark.internal.{LogKeys} import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute._ import org.apache.spark.ml.linalg._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala index cb9d8b32f0064..4bd1bfaf582d0 100755 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.feature import java.util.Locale import org.apache.spark.annotation.Since -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.ml.Transformer import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared.{HasInputCol, HasInputCols, HasOutputCol, HasOutputCols} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala index db3749558f473..6ba57f583ae31 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala @@ -23,7 +23,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.{SparkException, SparkIllegalArgumentException} import org.apache.spark.annotation.Since -import org.apache.spark.internal.{LogKeys, MDC} +import org.apache.spark.internal.{LogKeys} import org.apache.spark.ml.{Estimator, Model, Transformer} import org.apache.spark.ml.attribute.{Attribute, NominalAttribute} import org.apache.spark.ml.param._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala index 17adf61e46e9d..346595f80a9c4 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala @@ -17,13 +17,12 @@ package org.apache.spark.ml.optim +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{NUM_ITERATIONS, RELATIVE_TOLERANCE} -import org.apache.spark.internal.MDC import org.apache.spark.ml.feature.{Instance, OffsetInstance} import org.apache.spark.ml.linalg._ import org.apache.spark.ml.util.OptionalInstrumentation import org.apache.spark.rdd.RDD -import org.apache.spark.util.MavenUtils.LogStringContext /** * Model fitted by [[IterativelyReweightedLeastSquares]]. @@ -64,7 +63,7 @@ private[ml] class IterativelyReweightedLeastSquares( val fitIntercept: Boolean, val regParam: Double, val maxIter: Int, - val tol: Double) extends Serializable { + val tol: Double) extends Serializable with Logging { def fit( instances: RDD[OffsetInstance], diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala index 31db7b18efb3a..177f75b898c1b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala @@ -17,13 +17,12 @@ package org.apache.spark.ml.optim +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.COUNT -import org.apache.spark.internal.MDC import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.linalg._ import org.apache.spark.ml.util.OptionalInstrumentation import org.apache.spark.rdd.RDD -import org.apache.spark.util.MavenUtils.LogStringContext /** * Model fitted by [[WeightedLeastSquares]]. @@ -85,7 +84,7 @@ private[ml] class WeightedLeastSquares( val solverType: WeightedLeastSquares.Solver = WeightedLeastSquares.Auto, val maxIter: Int = 100, val tol: Double = 1e-6 - ) extends Serializable { + ) extends Serializable with Logging { import WeightedLeastSquares._ require(regParam >= 0.0, s"regParam cannot be negative: $regParam") diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/RWrapperUtils.scala b/mllib/src/main/scala/org/apache/spark/ml/r/RWrapperUtils.scala index 9e66647ef35fb..87948d1796d2f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/RWrapperUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/RWrapperUtils.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.r -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{FEATURE_COLUMN, LABEL_COLUMN, NEW_FEATURE_COLUMN_NAME, NEW_LABEL_COLUMN_NAME} import org.apache.spark.ml.attribute.{Attribute, AttributeGroup, NominalAttribute} import org.apache.spark.ml.feature.{RFormula, RFormulaModel} diff --git a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala index 276c7630d2d51..538ad03820754 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala @@ -33,7 +33,7 @@ import org.json4s.JsonDSL._ import org.apache.spark.{Partitioner, SparkException} import org.apache.spark.annotation.Since -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.PATH import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.linalg.BLAS diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index 3aee34a148adf..d96500ea84ab9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -26,7 +26,7 @@ import breeze.optimize.{CachedDiffFunction, LBFGS => BreezeLBFGS} import org.apache.hadoop.fs.Path import org.apache.spark.annotation.Since -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.ml.PredictorParams import org.apache.spark.ml.feature._ import org.apache.spark.ml.linalg._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala index 2c40a2f353b7d..f049e9a44cc28 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala @@ -22,7 +22,7 @@ import org.json4s.{DefaultFormats, JObject} import org.json4s.JsonDSL._ import org.apache.spark.annotation.Since -import org.apache.spark.internal.{LogKeys, MDC} +import org.apache.spark.internal.{LogKeys} import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param.ParamMap diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala index 163921c703903..c8fa97bfccce0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala @@ -21,7 +21,7 @@ import org.json4s.{DefaultFormats, JObject} import org.json4s.JsonDSL._ import org.apache.spark.annotation.Since -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.ml.linalg.{BLAS, Vector} import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala index ac47f81e2b1ec..f4d6e4e3562af 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkException import org.apache.spark.annotation.Since -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.ml.PredictorParams import org.apache.spark.ml.attribute._ import org.apache.spark.ml.feature.{Instance, OffsetInstance} diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala index 04e36c7f27ff3..a9e2c47a3229a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala @@ -21,7 +21,7 @@ import org.json4s.{DefaultFormats, JObject} import org.json4s.JsonDSL._ import org.apache.spark.annotation.Since -import org.apache.spark.internal.{LogKeys, MDC} +import org.apache.spark.internal.{LogKeys} import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param.ParamMap diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala index 2f63f4ae073e5..181168385011d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.tree.impl import scala.collection.mutable import scala.util.Try -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.tree.TreeEnsembleParams import org.apache.spark.mllib.tree.configuration.Algo._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala index d1cad44a15c8a..81ffa5c86a9fa 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml.tree.impl import org.apache.spark.broadcast.Broadcast -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.TIMER import org.apache.spark.ml.EstimatorUtils import org.apache.spark.ml.feature.Instance diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala index 118d6d7a063ac..cabbc497571b6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala @@ -21,7 +21,7 @@ import scala.collection.mutable import scala.util.Random import org.apache.spark.broadcast.Broadcast -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{MAX_MEMORY_SIZE, MEMORY_SIZE, NUM_CLASSES, NUM_EXAMPLES, NUM_FEATURES, NUM_NODES, NUM_WEIGHTED_EXAMPLES, TIMER} import org.apache.spark.ml.EstimatorUtils import org.apache.spark.ml.classification.DecisionTreeClassificationModel diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala index bc6b747344e31..119cc47a56f4f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala @@ -27,7 +27,7 @@ import org.apache.hadoop.fs.Path import org.json4s.DefaultFormats import org.apache.spark.annotation.Since -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CROSS_VALIDATION_METRIC, CROSS_VALIDATION_METRICS, ESTIMATOR_PARAM_MAP} import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.evaluation.Evaluator diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala index 324a08ba0b5ab..6ee64ef99a668 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala @@ -28,7 +28,7 @@ import org.apache.hadoop.fs.Path import org.json4s.DefaultFormats import org.apache.spark.annotation.Since -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{ESTIMATOR_PARAM_MAP, TRAIN_VALIDATION_SPLIT_METRIC, TRAIN_VALIDATION_SPLIT_METRICS} import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.evaluation.Evaluator diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/DatasetUtils.scala b/mllib/src/main/scala/org/apache/spark/ml/util/DatasetUtils.scala index d5b6396e5ba80..06de43260b303 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/DatasetUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/DatasetUtils.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml.util import org.apache.spark.SparkException -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CLASS_NAME, LABEL_COLUMN, NUM_CLASSES} import org.apache.spark.ml.PredictorParams import org.apache.spark.ml.classification.ClassifierParams diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/Instrumentation.scala b/mllib/src/main/scala/org/apache/spark/ml/util/Instrumentation.scala index 9413605a31ced..ddd63736023a3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/Instrumentation.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/Instrumentation.scala @@ -27,7 +27,7 @@ import org.json4s._ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.internal.{LogEntry, Logging, MDC} +import org.apache.spark.internal.{LogEntry, Logging} import org.apache.spark.internal.LogKeys.{CLASS_NAME, NUM_PARTITIONS, PIPELINE_STAGE_UID, STORAGE_LEVEL} import org.apache.spark.ml.{MLEvents, PipelineStage} import org.apache.spark.ml.param.{Param, Params} diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala b/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala index dd705219a4071..e3f31874a4c23 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala @@ -38,7 +38,7 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.annotation.{Since, Unstable} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.PATH import org.apache.spark.ml._ import org.apache.spark.ml.classification.{OneVsRest, OneVsRestModel} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala index c826654f0893c..0d94c76640291 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala @@ -24,7 +24,7 @@ import scala.collection.mutable import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CLUSTER_LEVEL, COST, DIVISIBLE_CLUSTER_INDICES_SIZE, FEATURE_DIMENSION, MIN_POINT_PER_CLUSTER, NUM_POINT} import org.apache.spark.ml.util.Instrumentation import org.apache.spark.mllib.linalg.{Vector, Vectors} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index 52fbc7a5a47f5..21c2af721428c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -21,7 +21,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.annotation.Since import org.apache.spark.broadcast.Broadcast -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{COST, INIT_MODE, NUM_ITERATIONS, TOTAL_TIME} import org.apache.spark.ml.util.Instrumentation import org.apache.spark.mllib.linalg.{Vector, Vectors} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala index ea83be1237298..ac31c0d3be479 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.clustering import scala.util.Random -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{NUM_ITERATIONS, POINT_OF_CENTER} import org.apache.spark.mllib.linalg.BLAS.{axpy, scal} import org.apache.spark.mllib.linalg.Vectors diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala index 639e762ef3c87..10e8a992c784e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala @@ -25,7 +25,7 @@ import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.graphx._ -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{DELTA, DIFF_DELTA, NORM, NUM_ITERATIONS} import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.{Loader, MLUtils, Saveable} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala index 641b4fa4048a6..c05ca9ecff3e9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala @@ -21,7 +21,7 @@ import scala.reflect.ClassTag import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaSparkContext._ -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.LogKeys.{CLUSTER_CENTROIDS, CLUSTER_LABEL, CLUSTER_WEIGHT, LARGEST_CLUSTER_INDEX, SMALLEST_CLUSTER_INDEX} import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala index a203d3fc73537..42007e96ca3a2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala @@ -18,7 +18,7 @@ package org.apache.spark.mllib.evaluation import org.apache.spark.annotation.Since -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{COUNT, NUM_BIN} import org.apache.spark.mllib.evaluation.binary._ import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index 100fa13db5180..d99dfd504cb80 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -31,7 +31,7 @@ import org.apache.spark.SparkContext import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.broadcast.Broadcast -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{ALPHA, COUNT, NUM_TRAIN_WORD, VOCAB_SIZE} import org.apache.spark.internal.config.Kryo.KRYO_SERIALIZER_MAX_BUFFER_SIZE import org.apache.spark.ml.linalg.BLAS diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala index 9c16ac2ecd526..df10fee3330bf 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala @@ -33,7 +33,7 @@ import org.apache.spark.SparkContext import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.java.JavaSparkContext.fakeClassTag -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{MIN_NUM_FREQUENT_PATTERN, NUM_FREQUENT_ITEMS, NUM_LOCAL_FREQUENT_PATTERN, NUM_PREFIXES, NUM_SEQUENCES} import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index 3329682d3b550..6c5e7bed4c671 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -23,7 +23,7 @@ import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, Matrix => BM} import org.apache.spark.{Partitioner, PartitionIdPassthrough, SparkException} import org.apache.spark.annotation.Since -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.mllib.linalg._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index 7621489bb1f0c..01abf22ad87c6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -26,7 +26,7 @@ import breeze.linalg.{axpy => brzAxpy, inv, svd => brzSvd, DenseMatrix => BDM, D import breeze.numerics.{sqrt => brzSqrt} import org.apache.spark.annotation.Since -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.config.MAX_RESULT_SIZE import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.stat._ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index d4ff30c6add04..c7ed8b377fdb0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -21,7 +21,7 @@ import scala.collection.mutable.ArrayBuffer import breeze.linalg.{norm, DenseVector => BDV} -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala index 3a8cde3ca1e89..2d3eab2a5708c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala @@ -22,7 +22,7 @@ import scala.collection.mutable import breeze.linalg.{DenseVector => BDV} import breeze.optimize.{CachedDiffFunction, DiffFunction, LBFGS => BreezeLBFGS} -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.linalg.BLAS.axpy import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index fb9e8ac7c8920..9d239b5b7a503 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -30,7 +30,7 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.SparkContext import org.apache.spark.annotation.Since import org.apache.spark.api.java.{JavaPairRDD, JavaRDD} -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.mllib.linalg.BLAS import org.apache.spark.mllib.rdd.MLPairRDDFunctions._ import org.apache.spark.mllib.util.{Loader, Saveable} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala index 2fa1339bc72a9..124a4a6596d27 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala @@ -21,7 +21,7 @@ import scala.reflect.ClassTag import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaSparkContext.fakeClassTag -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{MODEL_WEIGHTS, TIME} import org.apache.spark.mllib.linalg.Vector import org.apache.spark.streaming.api.java.{JavaDStream, JavaPairDStream} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala index 2059a9f785381..15f7a700be312 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.stat.test import org.apache.commons.math3.distribution.ChiSquaredDistribution import org.apache.spark.SparkException -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala index 2acc49e218f2d..299fd1073f8ed 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -26,7 +26,7 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.SparkContext import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.tree.configuration.{Algo, FeatureType} import org.apache.spark.mllib.tree.configuration.Algo._ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala index af922dda13f6b..8737cd1617be0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala @@ -26,7 +26,7 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.SparkContext import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.ml.linalg.BLAS import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala index 4857c9b00f421..d7ed3ab1e2abf 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala @@ -18,7 +18,7 @@ package org.apache.spark.mllib.util import org.apache.spark.annotation.Since -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{COUNT, RANGE} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index b8fcb1ffcbfe1..5a213e6803486 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.annotation.Since -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.OPTIMIZER_CLASS_NAME import org.apache.spark.ml.linalg.{MatrixUDT => MLMatrixUDT, VectorUDT => MLVectorUDT} import org.apache.spark.ml.util.Instrumentation diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala index 384fcf6ceb859..0c97e48079599 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkException -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CATEGORICAL_FEATURES, MAX_CATEGORIES} import org.apache.spark.ml.attribute._ import org.apache.spark.ml.linalg.{SparseVector, Vector, Vectors} diff --git a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala index 4da67a92d7075..e5ecd376c3590 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala @@ -28,7 +28,7 @@ import org.apache.commons.io.FileUtils import org.apache.commons.io.filefilter.TrueFileFilter import org.apache.spark._ -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{RMSE, TEST_SIZE, TRAINING_SIZE} import org.apache.spark.ml.linalg.{BLAS, Vectors} import org.apache.spark.ml.recommendation.ALS._ diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala index 135d7e26c6d8c..5f681317a58b6 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala @@ -26,7 +26,6 @@ import org.json4s.jackson.JsonMethods.{parse => parseJson} import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} import org.apache.spark.internal.LogKeys.MALFORMATTED_STRING -import org.apache.spark.internal.MDC import org.apache.spark.internal.config.Kryo._ import org.apache.spark.ml.{linalg => newlinalg} import org.apache.spark.mllib.util.TestingUtils._ diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala index e654eac83649c..4240da825ce57 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala @@ -18,8 +18,8 @@ package org.apache.spark.mllib.tree import org.apache.spark.SparkFunSuite -import org.apache.spark.internal.{MDC, MessageWithContext} import org.apache.spark.internal.LogKeys.{LEARNING_RATE, NUM_ITERATIONS, SUBSAMPLING_RATE} +import org.apache.spark.internal.MessageWithContext import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.{BoostingStrategy, Strategy} import org.apache.spark.mllib.tree.configuration.Algo._ diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala index cac3c13834a2a..182ba1fe15186 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala @@ -26,7 +26,7 @@ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.features.DriverServiceFeatureStep._ import org.apache.spark.deploy.k8s.submit._ -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.config.ConfigEntry import org.apache.spark.resource.ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID import org.apache.spark.util.{Clock, SystemClock, Utils} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala index 7f8bdb38b8936..5f79c894c2329 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala @@ -32,7 +32,7 @@ import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.annotation.{DeveloperApi, Since, Unstable} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.k8s.Config.KUBERNETES_FILE_UPLOAD_PATH -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.POD_ID import org.apache.spark.launcher.SparkLauncher import org.apache.spark.resource.ResourceUtils diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala index e09cd5cea64d4..e1b4969bfe799 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala @@ -29,7 +29,7 @@ import io.fabric8.kubernetes.client.utils.Utils.getSystemPropertyOrEnvVar import org.apache.spark.SparkConf import org.apache.spark.annotation.{DeveloperApi, Since, Stable} import org.apache.spark.deploy.k8s.Config._ -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.K8S_CONTEXT import org.apache.spark.internal.config.ConfigEntry diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala index a8706370eead6..f15f5bc566b4b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala @@ -24,7 +24,7 @@ import org.apache.spark.deploy.k8s._ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.submit._ -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CONFIG, CONFIG2, CONFIG3, CONFIG4, CONFIG5} import org.apache.spark.internal.config.{PYSPARK_DRIVER_PYTHON, PYSPARK_PYTHON} import org.apache.spark.launcher.SparkLauncher diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala index 042f984933a7e..a7a8e248f9c48 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala @@ -31,7 +31,7 @@ import org.apache.spark.deploy.k8s._ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.KubernetesUtils.addOwnerReference -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{APP_ID, APP_NAME, SUBMISSION_ID} import org.apache.spark.util.Utils diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientUtils.scala index d6b1da39bcbb5..729e45c3eb1da 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientUtils.scala @@ -32,7 +32,7 @@ import org.apache.spark.annotation.{DeveloperApi, Since, Unstable} import org.apache.spark.deploy.k8s.{Config, Constants, KubernetesUtils} import org.apache.spark.deploy.k8s.Config.{KUBERNETES_DNS_SUBDOMAIN_NAME_MAX_LENGTH, KUBERNETES_NAMESPACE} import org.apache.spark.deploy.k8s.Constants.ENV_SPARK_CONF_DIR -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CONFIG, PATH, PATHS} import org.apache.spark.util.ArrayImplicits._ diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala index 1bef7e50c0460..f4e205ee28f40 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala @@ -23,7 +23,7 @@ import io.fabric8.kubernetes.client.Watcher.Action import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.KubernetesDriverConf import org.apache.spark.deploy.k8s.KubernetesUtils._ -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{APP_ID, APP_NAME, POD_PHASE, POD_STATE, STATUS, SUBMISSION_ID} private[k8s] trait LoggingPodStatusWatcher extends Watcher[Pod] { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index 6021a4fb953e5..e84a0c97724c2 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -32,7 +32,7 @@ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.KubernetesConf import org.apache.spark.deploy.k8s.KubernetesUtils.addOwnerReference -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.config._ import org.apache.spark.resource.ResourceProfile import org.apache.spark.scheduler.cluster.SchedulerBackendUtils.DEFAULT_NUMBER_EXECUTORS diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala index fe2707a7f65b1..35386aff4a80c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala @@ -31,7 +31,7 @@ import org.apache.spark.deploy.ExecutorFailureTracker import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.KubernetesUtils._ -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.scheduler.ExecutorExited import org.apache.spark.util.{Clock, SystemClock, Utils} import org.apache.spark.util.SparkExitCode.EXCEED_MAX_EXECUTOR_FAILURES diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshot.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshot.scala index 44daf57d76ebf..2a8fd16d7e1ed 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshot.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshot.scala @@ -24,7 +24,7 @@ import io.fabric8.kubernetes.api.model.ContainerStateTerminated import io.fabric8.kubernetes.api.model.Pod import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{POD_NAME, POD_NAMESPACE, POD_PHASE} /** diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorRollPlugin.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorRollPlugin.scala index 548000893cd77..c624050d819ff 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorRollPlugin.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorRollPlugin.scala @@ -26,7 +26,7 @@ import org.apache.spark.SparkContext import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin} import org.apache.spark.deploy.k8s.Config.{EXECUTOR_ROLL_INTERVAL, EXECUTOR_ROLL_POLICY, ExecutorRollPolicy, MINIMUM_TASKS_PER_EXECUTOR_BEFORE_ROLLING} import org.apache.spark.executor.ExecutorMetrics -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CLASS_NAME, CONFIG, EXECUTOR_ID, INTERVAL} import org.apache.spark.internal.config.DECOMMISSION_ENABLED import org.apache.spark.scheduler.ExecutorDecommissionInfo diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index ddcdc2cf663ac..49eac64745b7c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -25,7 +25,7 @@ import org.apache.spark.{SparkConf, SparkContext, SparkMasterRegex} import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesUtils, SparkKubernetesClientFactory} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants.DEFAULT_EXECUTOR_CONTAINER_NAME -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.MASTER_URL import org.apache.spark.internal.config.TASK_MAX_FAILURES import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 09faa2a7fb1b3..aacd8b84199e1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -33,7 +33,6 @@ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.submit.KubernetesClientUtils import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.internal.LogKeys.{COUNT, TOTAL} -import org.apache.spark.internal.MDC import org.apache.spark.internal.config.SCHEDULER_MIN_REGISTERED_RESOURCES_RATIO import org.apache.spark.resource.ResourceProfile import org.apache.spark.rpc.{RpcAddress, RpcCallContext} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/shuffle/KubernetesLocalDiskShuffleExecutorComponents.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/shuffle/KubernetesLocalDiskShuffleExecutorComponents.scala index 2728385874f6d..cbe215c3f218b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/shuffle/KubernetesLocalDiskShuffleExecutorComponents.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/shuffle/KubernetesLocalDiskShuffleExecutorComponents.scala @@ -27,7 +27,7 @@ import org.apache.commons.io.FileExistsException import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.deploy.k8s.Config.KUBERNETES_DRIVER_REUSE_PVC -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.config.{SHUFFLE_CHECKSUM_ALGORITHM, SHUFFLE_CHECKSUM_ENABLED} import org.apache.spark.shuffle.ShuffleChecksumUtils.{compareChecksums, getChecksumFileName} import org.apache.spark.shuffle.api.{ShuffleExecutorComponents, ShuffleMapOutputWriter, SingleSpillShuffleMapOutputWriter} diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DepsTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DepsTestsSuite.scala index b80d42bb82256..785983d408163 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DepsTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DepsTestsSuite.scala @@ -38,7 +38,7 @@ import org.apache.spark.deploy.k8s.integrationtest.DepsTestsSuite.{DEPS_TIMEOUT, import org.apache.spark.deploy.k8s.integrationtest.KubernetesSuite._ import org.apache.spark.deploy.k8s.integrationtest.Utils.getExamplesJarName import org.apache.spark.deploy.k8s.integrationtest.backend.minikube.Minikube -import org.apache.spark.internal.{LogKeys, MDC} +import org.apache.spark.internal.{LogKeys} import org.apache.spark.internal.config.{ARCHIVES, PYSPARK_DRIVER_PYTHON, PYSPARK_PYTHON} private[spark] trait DepsTestsSuite { k8sSuite: KubernetesSuite => diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 3f8e97052d811..92c54fbc48601 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -41,7 +41,7 @@ import org.apache.spark.deploy.{ExecutorFailureTracker, SparkHadoopUtil} import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.deploy.yarn.config._ -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.config._ import org.apache.spark.internal.config.UI._ import org.apache.spark.metrics.{MetricsSystem, MetricsSystemInstances} diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 514bc41a7df1c..e742fd620f0c4 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -56,7 +56,7 @@ import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.deploy.yarn.ResourceRequestHelper._ import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ import org.apache.spark.deploy.yarn.config._ -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Python._ import org.apache.spark.launcher.{JavaModuleOptions, LauncherBackend, SparkAppHandle, YarnCommandBuilderUtils} diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 62753f35ae76c..858b4eef07455 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.yarn import scala.collection.mutable.ArrayBuffer -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.ARGS // TODO: Add code and support for ensuring that yarn resource 'tasks' are location aware ! diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 983ab5b4341b8..052b759e3f5c5 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -38,7 +38,7 @@ import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.Records import org.apache.spark.{SecurityManager, SparkConf, SparkException} -import org.apache.spark.internal.{Logging, MDC, MessageWithContext} +import org.apache.spark.internal.{Logging, MessageWithContext} import org.apache.spark.internal.LogKeys.{EXECUTOR_ENVS, EXECUTOR_LAUNCH_COMMANDS, EXECUTOR_RESOURCES} import org.apache.spark.internal.config._ import org.apache.spark.network.util.JavaUtils diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala index a747f99f1b85d..899382d4c5d13 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.yarn.exceptions.ResourceNotFoundException import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.deploy.yarn.config._ -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{ERROR, RESOURCE_NAME} import org.apache.spark.internal.config._ import org.apache.spark.resource.ResourceID diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala index c43b0e5388e68..b1bfbbb385747 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic import org.apache.hadoop.net._ import org.apache.hadoop.util.ReflectionUtils -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.NODE_LOCATION import org.apache.spark.util.SparkStringUtils diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 911ce2e27f96d..48946b5a8e6ea 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -40,7 +40,7 @@ import org.apache.spark.deploy.yarn.ResourceRequestHelper._ import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ import org.apache.spark.deploy.yarn.config._ import org.apache.spark.executor.ExecutorExitCode -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys import org.apache.spark.internal.config._ import org.apache.spark.resource.ResourceProfile diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorNodeHealthTracker.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorNodeHealthTracker.scala index 6938c0d7f8020..7455f33f48b0b 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorNodeHealthTracker.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorNodeHealthTracker.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest import org.apache.spark.SparkConf import org.apache.spark.deploy.ExecutorFailureTracker import org.apache.spark.deploy.yarn.config._ -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{FAILURES, HOST, NODES} import org.apache.spark.internal.config._ import org.apache.spark.scheduler.HealthTracker diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 8032d782cf4fc..9389a13e292f8 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.yarn.api.records.{FinalApplicationStatus, YarnApplicati import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.deploy.yarn.{Client, ClientArguments, YarnAppReport} import org.apache.spark.deploy.yarn.config._ -import org.apache.spark.internal.{config, Logging, LogKeys, MDC} +import org.apache.spark.internal.{config, Logging, LogKeys} import org.apache.spark.internal.LogKeys.{APP_ID, APP_STATE} import org.apache.spark.launcher.SparkAppHandle import org.apache.spark.scheduler.TaskSchedulerImpl diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala index cd81f11510fee..dae3317f0892c 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala @@ -29,7 +29,7 @@ import org.apache.hadoop.yarn.api.records.{ApplicationAttemptId, ApplicationId} import org.apache.spark.SparkContext import org.apache.spark.deploy.security.HadoopDelegationTokenManager -import org.apache.spark.internal.{config, Logging, MDC} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.LogKeys import org.apache.spark.internal.config.UI._ import org.apache.spark.resource.ResourceProfile diff --git a/sql/api/src/main/scala/org/apache/spark/sql/Column.scala b/sql/api/src/main/scala/org/apache/spark/sql/Column.scala index 88d597fdfbb73..316b629df4c14 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/Column.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql import scala.jdk.CollectionConverters._ import org.apache.spark.annotation.Stable -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{LEFT_EXPR, RIGHT_EXPR} import org.apache.spark.sql.catalyst.parser.DataTypeParser import org.apache.spark.sql.catalyst.trees.CurrentOrigin.withOrigin diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala index ab27d15d85409..f149fd5f165f7 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala @@ -27,7 +27,7 @@ import org.antlr.v4.runtime.misc.{Interval, ParseCancellationException} import org.antlr.v4.runtime.tree.TerminalNodeImpl import org.apache.spark.{QueryContext, SparkException, SparkThrowable, SparkThrowableHelper} -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, Origin, SQLQueryContext, WithOrigin} import org.apache.spark.sql.catalyst.util.SparkParserUtils diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/UDTRegistration.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/UDTRegistration.scala index 85d421a07577b..7bc9bde65c19b 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/UDTRegistration.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/UDTRegistration.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.types import scala.collection.mutable import org.apache.spark.annotation.{DeveloperApi, Since} -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.sql.errors.DataTypeErrors import org.apache.spark.util.SparkClassUtils diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatch.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatch.java index 5074348a1fd6a..4a041b70cf592 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatch.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatch.java @@ -129,7 +129,7 @@ private boolean acquirePage(long requiredSize) { page = allocatePage(requiredSize); } catch (SparkOutOfMemoryError e) { logger.warn("Failed to allocate page ({} bytes).", - MDC.of(LogKeys.PAGE_SIZE$.MODULE$, requiredSize)); + MDC.of(LogKeys.PAGE_SIZE, requiredSize)); return false; } base = page.getBaseObject(); diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/util/CaseInsensitiveStringMap.java b/sql/catalyst/src/main/java/org/apache/spark/sql/util/CaseInsensitiveStringMap.java index ec461f9740019..0c85f9b41cbaf 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/util/CaseInsensitiveStringMap.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/util/CaseInsensitiveStringMap.java @@ -63,7 +63,7 @@ public CaseInsensitiveStringMap(Map originalMap) { String key = toLowerCase(entry.getKey()); if (delegate.containsKey(key)) { logger.warn("Converting duplicated key {} into CaseInsensitiveStringMap.", - MDC.of(LogKeys.KEY$.MODULE$, entry.getKey())); + MDC.of(LogKeys.KEY, entry.getKey())); } delegate.put(key, entry.getValue()); } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 2cf85cb7e30c4..99e0c707d8871 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -24,7 +24,7 @@ import scala.collection.mutable import scala.reflect.ClassTag import org.apache.spark.SparkUnsupportedOperationException -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.FUNCTION_NAME import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.FunctionIdentifier diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HintErrorLogger.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HintErrorLogger.scala index 5301a3683c7da..8edc5278b5319 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HintErrorLogger.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HintErrorLogger.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.analysis -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{QUERY_HINT, RELATION_NAME, UNSUPPORTED_HINT_REASON} import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.logical.{HintErrorHandler, HintInfo} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala index 0e61b9cce681f..c4549a189e8e1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala @@ -21,7 +21,6 @@ import scala.util.control.NonFatal import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ -import org.apache.spark.internal.MDC import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, LogicalPlan} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala index 55b5b8ddfe6a3..fd4e081c91b52 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.analysis import java.util.Locale -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{ANALYSIS_ERROR, QUERY_PLAN} import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.ExtendedAnalysisException diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala index 4e35bd313d370..f88e2d112b558 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.analysis -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.SubExprUtils._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PlanLogger.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PlanLogger.scala index 3c9fb4bea4913..d1cd7adbe994a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PlanLogger.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PlanLogger.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.analysis.resolver -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{MESSAGE, QUERY_PLAN} import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index 0362ebf85bf1b..badfd0dfafb63 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -32,7 +32,7 @@ import org.json4s.JsonAST.{JArray, JBool, JDecimal, JDouble, JInt, JLong, JNull, import org.json4s.jackson.JsonMethods._ import org.apache.spark.SparkException -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{CurrentUserContext, FunctionIdentifier, InternalRow, SQLConfHelper, TableIdentifier} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVHeaderChecker.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVHeaderChecker.scala index 1c6eecad170f1..bec52747dea7c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVHeaderChecker.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVHeaderChecker.scala @@ -21,7 +21,7 @@ import com.univocity.parsers.common.AbstractParser import com.univocity.parsers.csv.{CsvParser, CsvParserSettings} import org.apache.spark.SparkIllegalArgumentException -import org.apache.spark.internal.{Logging, MDC, MessageWithContext} +import org.apache.spark.internal.{Logging, MessageWithContext} import org.apache.spark.internal.LogKeys.{CSV_HEADER_COLUMN_NAME, CSV_HEADER_COLUMN_NAMES, CSV_HEADER_LENGTH, CSV_SCHEMA_FIELD_NAME, CSV_SCHEMA_FIELD_NAMES, CSV_SOURCE, NUM_COLUMNS} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala index 53a5c44733986..fd3d1daae6742 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import java.lang.reflect.{Method, Modifier} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{FUNCTION_NAME, FUNCTION_PARAM} import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{InternalRow, SQLConfHelper} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 6b2c696fb9933..8c702815e9b9f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -33,7 +33,7 @@ import org.codehaus.janino.util.ClassFile.CodeAttribute import org.apache.spark.{SparkException, SparkIllegalArgumentException, TaskContext, TaskKilledException} import org.apache.spark.executor.InputMetrics -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.LogKeys._ import org.apache.spark.metrics.source.CodegenMetrics import org.apache.spark.sql.catalyst.InternalRow diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 4a828375f9d07..60f7145ed6e3d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.optimizer import scala.collection.mutable import org.apache.spark.SparkException -import org.apache.spark.internal.{LogKeys, MDC} +import org.apache.spark.internal.{LogKeys} import org.apache.spark.sql.catalyst.SQLConfHelper import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicate.scala index 0358c45815944..7134c3daf3baa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicate.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.SparkIllegalArgumentException import org.apache.spark.internal.LogKeys.{SQL_TEXT, UNSUPPORTED_EXPR} -import org.apache.spark.internal.MDC import org.apache.spark.sql.catalyst.expressions.{And, ArrayExists, ArrayFilter, CaseWhen, EqualNullSafe, Expression, If, In, InSet, LambdaFunction, Literal, MapFilter, Not, Or} import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLiteral} import org.apache.spark.sql.catalyst.plans.logical.{DeleteAction, DeleteFromTable, Filter, InsertAction, InsertStarAction, Join, LogicalPlan, MergeAction, MergeIntoTable, ReplaceData, UpdateAction, UpdateStarAction, UpdateTable, WriteDelta} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala index 9413bd7b454d4..be07b440a118b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.optimizer import scala.annotation.tailrec import scala.util.control.NonFatal -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{HASH_JOIN_KEYS, JOIN_CONDITION} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index b03a8811da308..0ed7bdb1f282f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -28,7 +28,7 @@ import org.antlr.v4.runtime.{ParserRuleContext, RuleContext, Token} import org.antlr.v4.runtime.tree.{ParseTree, RuleNode, TerminalNode} import org.apache.spark.{SparkArithmeticException, SparkException, SparkIllegalArgumentException, SparkThrowable, SparkThrowableHelper} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.PARTITION_SPECIFICATION import org.apache.spark.sql.catalyst.{EvaluateUnresolvedInlineTable, FunctionIdentifier, SQLConfHelper, TableIdentifier} import org.apache.spark.sql.catalyst.analysis._ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala index cc2d25ecf2dce..b5a35608a54e6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.catalyst.rules import org.apache.spark.SparkException import org.apache.spark.internal.{Logging, MessageWithContext} import org.apache.spark.internal.LogKeys._ -import org.apache.spark.internal.MDC import org.apache.spark.sql.catalyst.QueryPlanningTracker import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.util.DateTimeConstants.NANOS_PER_MILLIS diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CharVarcharUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CharVarcharUtils.scala index 467c04f1ed5b7..6fe35e08f88cf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CharVarcharUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CharVarcharUtils.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.util import scala.collection.mutable -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ParseMode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ParseMode.scala index cc1a01083af41..24662b8bbf4c6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ParseMode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ParseMode.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.util import java.util.Locale -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.PARSE_MODE sealed trait ParseMode { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala index 4efa7ae5a08c7..be17dc8ea470c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala @@ -21,7 +21,7 @@ import scala.collection.mutable.ArrayBuffer import scala.util.control.NonFatal import org.apache.spark.{SparkException, SparkUnsupportedOperationException} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{InternalRow, SQLConfHelper} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala index b4d737dcf791f..da83096903ad9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala @@ -24,7 +24,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.commons.text.similarity.LevenshteinDistance -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.errors.QueryCompilationErrors diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/V2ExpressionBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/V2ExpressionBuilder.scala index 5c89629ef10be..e42802de40f2f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/V2ExpressionBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/V2ExpressionBuilder.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.util -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.EXPR import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateFunction, Complete} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/xml/ValidatorUtil.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/xml/ValidatorUtil.scala index 8a5291d0bac74..dccbb40fdd985 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/xml/ValidatorUtil.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/xml/ValidatorUtil.scala @@ -28,7 +28,6 @@ import org.apache.spark.SparkFiles import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ -import org.apache.spark.internal.MDC /** * Utilities for working with XSD validation. diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/SparkSession.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/SparkSession.scala index 739b0318759e5..646db83981fe2 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/SparkSession.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/SparkSession.scala @@ -40,7 +40,7 @@ import org.apache.spark.api.java.JavaRDD import org.apache.spark.connect.proto import org.apache.spark.connect.proto.ExecutePlanResponse import org.apache.spark.connect.proto.ExecutePlanResponse.ObservedMetrics -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CONFIG, PATH} import org.apache.spark.rdd.RDD import org.apache.spark.sql diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/StreamingQueryListenerBus.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/StreamingQueryListenerBus.scala index 30cdf2b5cadb4..52b0ea24e9e33 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/StreamingQueryListenerBus.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/StreamingQueryListenerBus.scala @@ -22,7 +22,7 @@ import java.util.concurrent.CopyOnWriteArrayList import scala.jdk.CollectionConverters._ import org.apache.spark.connect.proto.{Command, ExecutePlanResponse, Plan, StreamingQueryEventType} -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.sql.connect.client.CloseableIterator import org.apache.spark.sql.streaming.StreamingQueryListener import org.apache.spark.sql.streaming.StreamingQueryListener.{Event, QueryIdleEvent, QueryProgressEvent, QueryStartedEvent, QueryTerminatedEvent} diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcRetryHandler.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcRetryHandler.scala index 0a38d18773deb..855d5651e3f70 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcRetryHandler.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcRetryHandler.scala @@ -23,7 +23,6 @@ import io.grpc.stub.StreamObserver import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{ERROR, NUM_RETRY, POLICY, RETRY_WAIT_TIME} -import org.apache.spark.internal.MDC private[sql] class GrpcRetryHandler( private val policies: Seq[RetryPolicy], diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala index 3a707495ff3f4..632a2aecee050 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala @@ -24,7 +24,7 @@ import io.grpc.stub.{ServerCallStreamObserver, StreamObserver} import org.apache.spark.{SparkEnv, SparkSQLException} import org.apache.spark.connect.proto.ExecutePlanResponse -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.catalyst.util.DateTimeConstants.NANOS_PER_MILLIS import org.apache.spark.sql.connect.common.ProtoUtils diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteResponseObserver.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteResponseObserver.scala index bcb665eb01efa..2473df0e53f1e 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteResponseObserver.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteResponseObserver.scala @@ -26,7 +26,7 @@ import io.grpc.stub.StreamObserver import org.apache.spark.{SparkEnv, SparkSQLException} import org.apache.spark.connect.proto -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys import org.apache.spark.sql.connect.config.Connect.CONNECT_EXECUTE_REATTACHABLE_OBSERVER_RETRY_BUFFER_SIZE import org.apache.spark.sql.connect.service.ExecuteHolder diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala index 93853805b4371..7c4ad7df66fc8 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala @@ -26,7 +26,7 @@ import com.google.protobuf.Message import org.apache.spark.SparkSQLException import org.apache.spark.connect.proto -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.sql.connect.common.ProtoUtils import org.apache.spark.sql.connect.planner.InvalidInputErrors import org.apache.spark.sql.connect.service.{ExecuteHolder, ExecuteSessionTag, SparkConnectService} diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 7320c6e3918c8..724526807211d 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -41,7 +41,7 @@ import org.apache.spark.connect.proto.ExecutePlanResponse.SqlCommandResult import org.apache.spark.connect.proto.Parse.ParseFormat import org.apache.spark.connect.proto.StreamingQueryManagerCommandResult.StreamingQueryInstance import org.apache.spark.connect.proto.WriteStreamOperationStart.TriggerCase -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.LogKeys.{DATAFRAME_ID, SESSION_ID} import org.apache.spark.resource.{ExecutorResourceRequest, ResourceProfile, TaskResourceProfile, TaskResourceRequest} import org.apache.spark.sql.{Column, Encoders, ForeachWriter, Observation, Row} diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala index 70f18a1a9c1a0..04312a35a3b4b 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala @@ -24,7 +24,7 @@ import io.grpc.stub.StreamObserver import org.apache.spark.connect.proto.ExecutePlanResponse import org.apache.spark.connect.proto.StreamingQueryListenerBusCommand import org.apache.spark.connect.proto.StreamingQueryListenerEventsResult -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.sql.connect.service.ExecuteHolder /** diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingForeachBatchHelper.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingForeachBatchHelper.scala index 5a44607a4c7d9..a4da5ea998386 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingForeachBatchHelper.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingForeachBatchHelper.scala @@ -26,7 +26,7 @@ import scala.util.control.NonFatal import org.apache.spark.SparkException import org.apache.spark.api.python.{PythonException, PythonWorkerUtils, SimplePythonFunction, SpecialLengths, StreamingPythonRunner} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{DATAFRAME_ID, PYTHON_EXEC, QUERY_ID, RUN_ID_STRING, SESSION_ID, USER_ID} import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, AgnosticEncoders} diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingQueryListenerHelper.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingQueryListenerHelper.scala index 42c090d43f065..f994ada920ec2 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingQueryListenerHelper.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingQueryListenerHelper.scala @@ -21,7 +21,7 @@ import java.io.EOFException import org.apache.spark.SparkException import org.apache.spark.api.python.{PythonException, PythonWorkerUtils, SimplePythonFunction, SpecialLengths, StreamingPythonRunner} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.FUNCTION_NAME import org.apache.spark.sql.connect.config.Connect import org.apache.spark.sql.connect.service.{SessionHolder, SparkConnectService} diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/LoggingInterceptor.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/LoggingInterceptor.scala index a071579692fb1..695034954e981 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/LoggingInterceptor.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/LoggingInterceptor.scala @@ -28,7 +28,7 @@ import io.grpc.ServerCall import io.grpc.ServerCallHandler import io.grpc.ServerInterceptor -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{DESCRIPTION, MESSAGE} /** diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala index 1b43ea529ec02..b7d8a6566cf9b 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala @@ -31,7 +31,7 @@ import com.google.common.cache.{Cache, CacheBuilder} import org.apache.spark.{SparkEnv, SparkException, SparkSQLException} import org.apache.spark.api.python.PythonFunction.PythonAccumulator import org.apache.spark.connect.proto -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.sql.DataFrame import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.classic.SparkSession diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectExecutionManager.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectExecutionManager.scala index c8f4a1bc977f8..35c4073fe93c9 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectExecutionManager.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectExecutionManager.scala @@ -30,7 +30,7 @@ import io.grpc.stub.StreamObserver import org.apache.spark.{SparkEnv, SparkSQLException} import org.apache.spark.connect.proto -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.sql.catalyst.util.DateTimeConstants.NANOS_PER_MILLIS import org.apache.spark.sql.connect.config.Connect.{CONNECT_EXECUTE_MANAGER_ABANDONED_TOMBSTONES_SIZE, CONNECT_EXECUTE_MANAGER_DETACHED_TIMEOUT, CONNECT_EXECUTE_MANAGER_MAINTENANCE_INTERVAL} import org.apache.spark.sql.connect.execution.ExecuteGrpcResponseSender diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectListenerBusListener.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectListenerBusListener.scala index 8fbcf3218a003..91fe395f520d8 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectListenerBusListener.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectListenerBusListener.scala @@ -29,7 +29,7 @@ import org.apache.spark.connect.proto.ExecutePlanResponse import org.apache.spark.connect.proto.StreamingQueryEventType import org.apache.spark.connect.proto.StreamingQueryListenerEvent import org.apache.spark.connect.proto.StreamingQueryListenerEventsResult -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.sql.connect.execution.ExecuteResponseObserver import org.apache.spark.sql.streaming.StreamingQueryListener import org.apache.spark.util.ArrayImplicits._ diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectServer.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectServer.scala index b2c4d1abb17b4..5b1034a4a27b7 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectServer.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectServer.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.connect.service -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{HOST, PORT} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.internal.SQLConf diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala index 04030c144c2e2..cc1cb95b66c46 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala @@ -34,7 +34,7 @@ import org.apache.spark.{SparkContext, SparkEnv} import org.apache.spark.connect.proto import org.apache.spark.connect.proto.{AddArtifactsRequest, AddArtifactsResponse, SparkConnectServiceGrpc} import org.apache.spark.connect.proto.SparkConnectServiceGrpc.AsyncService -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.HOST import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.scheduler.{LiveListenerBus, SparkListenerEvent} diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManager.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManager.scala index 572d760187e9d..1c3cfd67f132c 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManager.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManager.scala @@ -28,7 +28,7 @@ import scala.util.control.NonFatal import com.google.common.cache.CacheBuilder import org.apache.spark.{SparkEnv, SparkSQLException} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{INTERVAL, SESSION_HOLD_INFO} import org.apache.spark.sql.classic.SparkSession import org.apache.spark.sql.connect.config.Connect.{CONNECT_SESSION_MANAGER_CLOSED_SESSIONS_TOMBSTONES_SIZE, CONNECT_SESSION_MANAGER_DEFAULT_SESSION_TIMEOUT, CONNECT_SESSION_MANAGER_MAINTENANCE_INTERVAL} diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectStreamingQueryCache.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectStreamingQueryCache.scala index beff193f6701f..8ff13f5afe191 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectStreamingQueryCache.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectStreamingQueryCache.scala @@ -25,7 +25,7 @@ import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration.{Duration, DurationInt, FiniteDuration} import scala.util.control.NonFatal -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{DURATION, NEW_VALUE, OLD_VALUE, QUERY_CACHE_VALUE, QUERY_ID, QUERY_RUN_ID, SESSION_ID} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.streaming.StreamingQuery diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ui/SparkConnectServerListener.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ui/SparkConnectServerListener.scala index 3a93bbae3f2b8..98dccc6c9a6c8 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ui/SparkConnectServerListener.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ui/SparkConnectServerListener.scala @@ -23,7 +23,7 @@ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import org.apache.spark.{SparkConf, SparkContext, SparkEnv} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{OP_ID, SESSION_ID} import org.apache.spark.internal.config.Status.LIVE_ENTITY_UPDATE_PERIOD import org.apache.spark.scheduler._ diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/ErrorUtils.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/ErrorUtils.scala index 292d2eee0f159..47cf5e51bdb7b 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/ErrorUtils.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/ErrorUtils.scala @@ -37,7 +37,7 @@ import org.json4s.jackson.JsonMethods import org.apache.spark.{QueryContextType, SparkEnv, SparkException, SparkThrowable} import org.apache.spark.api.python.PythonException import org.apache.spark.connect.proto.FetchErrorDetailsResponse -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{OP_TYPE, SESSION_ID, USER_ID} import org.apache.spark.sql.connect.config.Connect import org.apache.spark.sql.connect.service.{ExecuteEventsManager, SessionHolder, SessionKey, SparkConnectService} diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/ProtoToParsedPlanTestSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/ProtoToParsedPlanTestSuite.scala index 8c1f50e661924..02c5683e02765 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/ProtoToParsedPlanTestSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/ProtoToParsedPlanTestSuite.scala @@ -27,7 +27,6 @@ import scala.util.{Failure, Success, Try} import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.connect.proto import org.apache.spark.internal.LogKeys.PATH -import org.apache.spark.internal.MDC import org.apache.spark.sql.catalyst.{catalog, QueryPlanningTracker} import org.apache.spark.sql.catalyst.analysis.{caseSensitiveResolution, Analyzer, FunctionRegistry, Resolver, TableFunctionRegistry} import org.apache.spark.sql.catalyst.catalog.SessionCatalog diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala index 40779c66600fd..2311bb736cfba 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala @@ -23,7 +23,7 @@ import java.nio.channels.{Channels, SocketChannel} import net.razorvine.pickle.{Pickler, Unpickler} import org.apache.spark.api.python.DechunkedInputStream -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.CLASS_LOADER import org.apache.spark.security.SocketAuthServer import org.apache.spark.sql.{internal, Column, DataFrame, Row, SparkSession} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala index 1efd8f9e32208..3fc3cf698df9b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala @@ -27,7 +27,7 @@ import org.apache.spark.TaskContext import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.api.r.SerDe import org.apache.spark.broadcast.Broadcast -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.CONFIG import org.apache.spark.rdd.RDD import org.apache.spark.sql.Row diff --git a/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala index 98e045bcb295b..b0efd09d362a7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala @@ -32,7 +32,7 @@ import org.apache.commons.io.FilenameUtils import org.apache.hadoop.fs.{LocalFileSystem, Path => FSPath} import org.apache.spark.{JobArtifactSet, JobArtifactState, SparkContext, SparkEnv, SparkException, SparkRuntimeException, SparkUnsupportedOperationException} -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.config.{CONNECT_SCALA_UDF_STUB_PREFIXES, EXECUTOR_USER_CLASS_PATH_FIRST} import org.apache.spark.sql.Artifact import org.apache.spark.sql.classic.SparkSession diff --git a/sql/core/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala index b388c98ffcb1d..c338bd428bbed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala @@ -31,7 +31,7 @@ import org.apache.hadoop.fs.FileStatus import org.apache.hadoop.mapreduce.Job import org.apache.spark.{SparkException, SparkIllegalArgumentException} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CODEC_LEVEL, CODEC_NAME, CONFIG, PATH} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.avro.AvroCompressionCodec._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala b/sql/core/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala index c936099ef3764..b425f63d6a7ed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala @@ -28,7 +28,7 @@ import org.apache.avro.Schema.Type._ import org.apache.avro.SchemaBuilder.FieldAssembler import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{FIELD_NAME, FIELD_TYPE, RECURSIVE_DEPTH} import org.apache.spark.sql.avro.AvroOptions.RECURSIVE_FIELD_MAX_DEPTH_LIMIT import org.apache.spark.sql.catalyst.parser.CatalystSqlParser diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index e51b04bad1df9..b664f52c8c1c5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.SparkException import org.apache.spark.internal.LogKeys.CONFIG -import org.apache.spark.internal.MDC import org.apache.spark.sql.SaveMode import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, CatalogUtils, ClusterBySpec} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala index 0015d7ff99e2b..681e1b16af590 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala @@ -31,7 +31,7 @@ import scala.util.control.NonFatal import org.apache.spark.{SPARK_VERSION, SparkConf, SparkContext, SparkException, TaskContext} import org.apache.spark.annotation.{DeveloperApi, Experimental, Stable, Unstable} import org.apache.spark.api.java.JavaRDD -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CALL_SITE_LONG_FORM, CLASS_NAME, CONFIG} import org.apache.spark.internal.config.{ConfigEntry, EXECUTOR_ALLOW_SPARK_CONTEXT} import org.apache.spark.rdd.RDD diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/StreamingQueryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/StreamingQueryManager.scala index 6d4a3ecd36037..5d2b74411b9c5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/StreamingQueryManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/StreamingQueryManager.scala @@ -25,7 +25,7 @@ import scala.collection.mutable import scala.jdk.CollectionConverters._ import org.apache.spark.annotation.Evolving -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CLASS_NAME, QUERY_ID, RUN_ID} import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.streaming.{WriteToStream, WriteToStreamStatement} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala index 6513e2cc829cb..2f1b79ec90f99 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.columnar import org.apache.spark.annotation.{DeveloperApi, Since} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{FILTER, PREDICATE} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala index 60156bff1fb71..bfd813ad5ef11 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala @@ -27,7 +27,7 @@ import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration import org.apache.spark.{SparkFiles, TaskContext} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index 5b68be2c4ce55..a8292a8dbaa3b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.spark.internal.{Logging, MDC, MessageWithContext} +import org.apache.spark.internal.{Logging, MessageWithContext} import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.catalyst.catalog.HiveTableRelation import org.apache.spark.sql.catalyst.expressions.{Attribute, SubqueryExpression} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 9f4a1718d820c..1c5dd572edfbc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -22,7 +22,6 @@ import java.util.concurrent.TimeUnit._ import org.apache.hadoop.fs.Path import org.apache.spark.internal.LogKeys.{COUNT, MAX_SPLIT_BYTES, OPEN_COST_IN_BYTES} -import org.apache.spark.internal.MDC import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.{FileSourceOptions, InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.BucketSpec diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index 6000b3c20251e..6148fb30783e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{LOGICAL_PLAN_COLUMNS, OPTIMIZED_PLAN_COLUMNS} import org.apache.spark.rdd.RDD import org.apache.spark.sql.Encoder diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArray.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArray.scala index ebf974a8a4800..3e98c28b29fbc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArray.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArray.scala @@ -22,7 +22,7 @@ import java.io.Closeable import scala.collection.mutable.ArrayBuffer import org.apache.spark.{SparkEnv, TaskContext} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CLASS_NAME, MAX_NUM_ROWS_IN_MEMORY_BUFFER} import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.serializer.SerializerManager diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 9e5264d8d4f31..86b67953750d2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -26,7 +26,7 @@ import scala.util.control.NonFatal import org.apache.hadoop.fs.Path import org.apache.spark.SparkException -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.EXTENDED_EXPLAIN_GENERATOR import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, ExtendedExplainGenerator, Row} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 178f867c32b58..246508965d3d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -25,7 +25,6 @@ import scala.util.control.NonFatal import org.apache.spark.{broadcast, SparkException, SparkUnsupportedOperationException} import org.apache.spark.internal.LogKeys.{CODEGEN_STAGE_ID, CONFIG, ERROR, HUGE_METHOD_LIMIT, MAX_METHOD_CODE_SIZE, TREE_NODE} -import org.apache.spark.internal.MDC import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEOptimizer.scala index 0f1743eeaacfb..f16c6d9cfe6dd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEOptimizer.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.adaptive import org.apache.spark.internal.LogKeys.{BATCH_NAME, RULE_NAME} -import org.apache.spark.internal.MDC import org.apache.spark.sql.catalyst.analysis.UpdateAttributeNullability import org.apache.spark.sql.catalyst.optimizer.{ConvertToLocalRelation, EliminateLimits, OptimizeOneRowPlan} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LogicalPlanIntegrity} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 00376670956d9..0e50c03b6cc9f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -28,8 +28,8 @@ import scala.util.control.NonFatal import org.apache.spark.SparkException import org.apache.spark.broadcast -import org.apache.spark.internal.{MDC, MessageWithContext} import org.apache.spark.internal.LogKeys._ +import org.apache.spark.internal.MessageWithContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala index 2855f902a8509..aa748d8de6dc0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.execution.adaptive import scala.collection.mutable import org.apache.spark.internal.LogKeys.{CONFIG, SUB_QUERY} -import org.apache.spark.internal.MDC import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions.{DynamicPruningSubquery, ListQuery, SubqueryExpression} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala index 1ea4df0254673..b7cf0ce3150ba 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.adaptive import scala.collection.mutable.ArrayBuffer import org.apache.spark.{MapOutputStatistics, MapOutputTrackerMaster, SparkEnv} -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.sql.execution.{CoalescedPartitionSpec, PartialReducerPartitionSpec, ShufflePartitionSpec} object ShufflePartitionsUtil extends Logging { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregateCodegenSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregateCodegenSupport.scala index 40112979c6d46..595cecabe9185 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregateCodegenSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregateCodegenSupport.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.execution.aggregate import org.apache.spark.SparkException import org.apache.spark.internal.LogKeys.MAX_JVM_METHOD_PARAMS_LENGTH -import org.apache.spark.internal.MDC import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, Expression, ExpressionEquals, UnsafeRow} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index 24528b6f4da15..4a0db1c613d60 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -23,7 +23,6 @@ import scala.collection.mutable import org.apache.spark.TaskContext import org.apache.spark.internal.LogKeys.CONFIG -import org.apache.spark.internal.MDC import org.apache.spark.memory.SparkOutOfMemoryError import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationIterator.scala index b89b268dd3c0f..fbd3ce040e487 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationIterator.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.aggregate import org.apache.spark.{SparkEnv, SparkException, TaskContext} -import org.apache.spark.internal.{config, Logging, MDC} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.LogKeys.{CONFIG, HASH_MAP_SIZE, OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTablesCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTablesCommand.scala index 26192551632e3..f3a0da2437ae8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTablesCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTablesCommand.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.execution.command import scala.util.control.NonFatal import org.apache.spark.internal.LogKeys.{DATABASE_NAME, ERROR, TABLE_NAME} -import org.apache.spark.internal.MDC import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.classic.ClassicConversions.castToImpl diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala index 7cbba170cd1e6..35cffc3a4dc96 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala @@ -24,7 +24,7 @@ import scala.util.control.NonFatal import org.apache.hadoop.fs.{FileStatus, FileSystem, Path, PathFilter} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{COUNT, DATABASE_NAME, ERROR, TABLE_NAME, TIME} import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.ResolvedIdentifier diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala index d3a72f915c47b..667623fb95b03 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.command import org.apache.spark.internal.LogKeys._ -import org.apache.spark.internal.MDC import org.apache.spark.sql.{AnalysisException, Row, SaveMode, SparkSession} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.plans.logical.{CTEInChildren, CTERelationDef, LogicalPlan, WithCTE} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala index 87cd9376b77b1..14fc72de3146d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.command -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CONFIG, CONFIG2, KEY, VALUE} import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.expressions.Attribute diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index f29d2267f75fd..785dff243f7e6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.execution.command import java.net.URI import org.apache.spark.internal.LogKeys._ -import org.apache.spark.internal.MDC import org.apache.spark.sql.{AnalysisException, Row, SaveMode, SparkSession} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.plans.logical.{CTEInChildren, CTERelationDef, LogicalPlan, WithCTE} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 2969194ae2f4b..7df687a4963e5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -28,7 +28,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ import org.apache.hadoop.mapred.{FileInputFormat, JobConf} -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.config.RDD_PARALLEL_LISTING_THRESHOLD import org.apache.spark.sql.{Row, SaveMode, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/BasicWriteStatsTracker.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/BasicWriteStatsTracker.scala index 1858a84213598..9a38cc35b33f8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/BasicWriteStatsTracker.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/BasicWriteStatsTracker.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.{SparkContext, TaskContext} -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.LogKeys.{ACTUAL_NUM_FILES, EXPECTED_NUM_FILES} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.SQLExecution diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 489f62bd97769..1f9abf6e3a965 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -27,7 +27,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkException import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CLASS_NAME, DATA_SOURCE, DATA_SOURCES, PATHS} import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession} import org.apache.spark.sql.catalyst.DataSourceOptions diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceManager.scala index 7a8dbab35964f..790797e1f3183 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceManager.scala @@ -21,7 +21,7 @@ import java.util.Locale import java.util.concurrent.ConcurrentHashMap import org.apache.spark.api.python.PythonUtils -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.DATA_SOURCE import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.execution.datasources.v2.python.UserDefinedPythonDataSource diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 882012c968e27..6bc793f892788 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -25,7 +25,7 @@ import scala.jdk.CollectionConverters._ import org.apache.hadoop.fs.Path -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.PREDICATES import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Row, SaveMode} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala index 374b29e4c1a06..e11c2b15e0541 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala @@ -22,7 +22,7 @@ import org.apache.hadoop.fs.{FileAlreadyExistsException, Path} import org.apache.hadoop.mapreduce.TaskAttemptContext import org.apache.spark.TaskOutputFileAlreadyExistException -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CONFIG, NUM_CONCURRENT_WRITER} import org.apache.spark.internal.io.{FileCommitProtocol, FileNameSpec} import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala index b4cffa59c98d5..55e2271dc058b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl import org.apache.spark._ -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.io.{FileCommitProtocol, SparkHadoopWriterUtils} import org.apache.spark.sql.catalyst.InternalRow diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala index 50af845c37cb1..68a9bbfc7ab1d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala @@ -21,7 +21,7 @@ import scala.collection.mutable.ArrayBuffer import scala.math.BigDecimal.RoundingMode import org.apache.spark.Partition -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CONFIG, DESIRED_NUM_PARTITIONS, MAX_NUM_PARTITIONS, NUM_PARTITIONS} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.classic.ClassicConversions._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index e468807f4ffd1..f874de8c1abdd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -27,7 +27,6 @@ import org.apache.hadoop.security.AccessControlException import org.apache.spark.{Partition => RDDPartition, TaskContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.LogKeys.{CURRENT_FILE, PATH} -import org.apache.spark.internal.MDC import org.apache.spark.paths.SparkPath import org.apache.spark.rdd.{InputFileBlockHolder, RDD} import org.apache.spark.sql.SparkSession diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index 7291da248294a..396375890c249 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -21,7 +21,7 @@ import java.util.Locale import scala.collection.mutable -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{NUM_PRUNED, POST_SCAN_FILTERS, PUSHED_FILTERS, TOTAL} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.catalog.BucketSpec diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileStatusCache.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileStatusCache.scala index 2bb8476a9f0e8..cf182ec18f1c9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileStatusCache.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileStatusCache.scala @@ -25,7 +25,7 @@ import scala.jdk.CollectionConverters._ import com.google.common.cache._ import org.apache.hadoop.fs.{FileStatus, Path} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CACHED_TABLE_PARTITION_METADATA_SIZE, MAX_TABLE_PARTITION_METADATA_SIZE} import org.apache.spark.sql.SparkSession import org.apache.spark.util.SizeEstimator diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala index 899adcae87ef2..03ddc7d609346 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala @@ -24,7 +24,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ import org.apache.hadoop.mapred.{FileInputFormat, JobConf} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{COUNT, ELAPSED_TIME} import org.apache.spark.metrics.source.HiveCatalogMetrics import org.apache.spark.sql.SparkSession diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala index 07be3f89872cc..1bf0d2f0301f2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala @@ -24,7 +24,7 @@ import scala.collection.mutable import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{COUNT, PERCENT, TOTAL} import org.apache.spark.paths.SparkPath import org.apache.spark.sql.SparkSession diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SQLHadoopMapReduceCommitProtocol.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SQLHadoopMapReduceCommitProtocol.scala index 03e988eb0bd2b..db9a65a3814db 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SQLHadoopMapReduceCommitProtocol.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SQLHadoopMapReduceCommitProtocol.scala @@ -21,7 +21,7 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.{OutputCommitter, TaskAttemptContext} import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.CLASS_NAME import org.apache.spark.internal.io.HadoopMapReduceCommitProtocol import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala index c6b9764bee2c9..596edc8beaa34 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala @@ -30,7 +30,7 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat import org.apache.spark.TaskContext import org.apache.spark.input.{PortableDataStream, StreamInputFormat} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.PATH import org.apache.spark.paths.SparkPath import org.apache.spark.rdd.{BinaryFileRDD, RDD} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala index 0ae5f0c142356..a0045ebb8a006 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala @@ -23,7 +23,7 @@ import scala.util.Using import scala.util.control.NonFatal import org.apache.spark.{InterruptibleIterator, Partition, SparkContext, SparkException, TaskContext} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.SQL_TEXT import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala index e1ef86f54eb12..05e30207314a7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala @@ -21,7 +21,7 @@ import scala.collection.mutable.ArrayBuffer import scala.math.BigDecimal.RoundingMode import org.apache.spark.Partition -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CLAUSES, LOWER_BOUND, NEW_VALUE, NUM_PARTITIONS, OLD_VALUE, UPPER_BOUND} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Row, SaveMode, SparkSession, SQLContext} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index 6a97c4e0cf46a..926c133d2875c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -31,7 +31,7 @@ import scala.util.control.NonFatal import org.apache.spark.{SparkContext, SparkThrowable, SparkUnsupportedOperationException, TaskContext} import org.apache.spark.executor.InputMetrics -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{DEFAULT_ISOLATION_LEVEL, ISOLATION_LEVEL} import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.catalyst.{InternalRow, SQLConfHelper} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonOutputWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonOutputWriter.scala index 5727c502a7097..4abfd96993785 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonOutputWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonOutputWriter.scala @@ -21,7 +21,7 @@ import java.nio.charset.{Charset, StandardCharsets} import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.TaskAttemptContext -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{ENCODING, PATH} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.json.{JacksonGenerator, JSONOptions, JSONOptionsInRead} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala index 50c28c783b4cd..20c40f65b0686 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala @@ -31,7 +31,7 @@ import org.apache.orc.{BooleanColumnStatistics, ColumnStatistics, DateColumnStat import org.apache.spark.{SPARK_VERSION_SHORT, SparkException} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.PATH import org.apache.spark.sql.{SPARK_VERSION_METADATA_KEY, SparkSession} import org.apache.spark.sql.catalyst.{FileSourceOptions, InternalRow} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index c71d55a41c764..661be2b9cfa08 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -32,7 +32,7 @@ import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GRO import org.apache.parquet.hadoop._ import org.apache.spark.TaskContext -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{PATH, SCHEMA} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala index bad883534115c..5db3b0671db9f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala @@ -33,7 +33,7 @@ import org.apache.parquet.schema.{PrimitiveType, Types} import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName import org.apache.spark.{SparkException, SparkUnsupportedOperationException} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CLASS_NAME, CONFIG} import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CacheTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CacheTableExec.scala index 2e295200e9ce9..a28b40dc7cbf5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CacheTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CacheTableExec.scala @@ -22,7 +22,6 @@ import java.util.Locale import scala.util.control.NonFatal import org.apache.spark.internal.LogKeys.OPTIONS -import org.apache.spark.internal.MDC import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{LocalTempView, UnresolvedRelation} import org.apache.spark.sql.catalyst.expressions.Attribute diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateIndexExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateIndexExec.scala index 60d44101da3b1..1de97ba4189ea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateIndexExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateIndexExec.scala @@ -22,7 +22,6 @@ import java.util import scala.jdk.CollectionConverters._ import org.apache.spark.internal.LogKeys.{INDEX_NAME, TABLE_NAME} -import org.apache.spark.internal.MDC import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.IndexAlreadyExistsException import org.apache.spark.sql.catalyst.expressions.Attribute diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateNamespaceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateNamespaceExec.scala index 76ba53ef99a00..ef069fd2fe23b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateNamespaceExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateNamespaceExec.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.jdk.CollectionConverters.MapHasAsJava import org.apache.spark.internal.LogKeys.NAMESPACE -import org.apache.spark.internal.MDC import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.NamespaceAlreadyExistsException import org.apache.spark.sql.catalyst.expressions.Attribute diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala index 3298a56d95992..58c63be38833f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.jdk.CollectionConverters._ import org.apache.spark.internal.LogKeys.TABLE_NAME -import org.apache.spark.internal.MDC import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException import org.apache.spark.sql.catalyst.expressions.Attribute diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 9542a6f4ff10e..6579822215867 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -22,7 +22,7 @@ import scala.collection.mutable import org.apache.hadoop.fs.Path import org.apache.spark.SparkException -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.EXPR import org.apache.spark.sql.catalyst.analysis.{ResolvedIdentifier, ResolvedNamespace, ResolvedPartitionSpec, ResolvedTable} import org.apache.spark.sql.catalyst.catalog.CatalogUtils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIndexExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIndexExec.scala index 4fe6c3cd4a0e8..b10f59686062e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIndexExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIndexExec.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.internal.LogKeys.INDEX_NAME -import org.apache.spark.internal.MDC import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.NoSuchIndexException import org.apache.spark.sql.catalyst.expressions.Attribute diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileBatchWrite.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileBatchWrite.scala index b9f058b55ed02..345a0b0a4a290 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileBatchWrite.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileBatchWrite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.hadoop.mapreduce.Job -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.io.FileCommitProtocol import org.apache.spark.sql.connector.write.{BatchWrite, DataWriterFactory, PhysicalWriteInfo, WriterCommitMessage} import org.apache.spark.sql.execution.datasources.{WriteJobDescription, WriteTaskResult} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala index be4f5dcb65aa1..d1f445ec63d93 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala @@ -21,7 +21,7 @@ import java.io.{FileNotFoundException, IOException} import org.apache.hadoop.hdfs.BlockMissingException import org.apache.hadoop.security.AccessControlException -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CURRENT_FILE, PARTITIONED_FILE_READER} import org.apache.spark.rdd.InputFileBlockHolder import org.apache.spark.sql.catalyst.FileSourceOptions diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala index 3663d6a98b864..5348f9ab6df62 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala @@ -20,7 +20,7 @@ import java.util.{Locale, OptionalLong} import org.apache.hadoop.fs.Path -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{PATH, REASON} import org.apache.spark.internal.config.IO_WARNING_LARGEFILETHRESHOLD import org.apache.spark.sql.SparkSession diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/GroupBasedRowLevelOperationScanPlanning.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/GroupBasedRowLevelOperationScanPlanning.scala index 8b8cdc06d398b..77e21bcbbdb8f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/GroupBasedRowLevelOperationScanPlanning.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/GroupBasedRowLevelOperationScanPlanning.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.datasources.v2 -import org.apache.spark.internal.{LogKeys, MDC} +import org.apache.spark.internal.{LogKeys} import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, AttributeSet, Expression, ExpressionSet, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral import org.apache.spark.sql.catalyst.planning.{GroupBasedRowLevelOperation, PhysicalOperation} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanPartitioningAndOrdering.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanPartitioningAndOrdering.scala index 7ee4374b45789..5d06c8786d894 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanPartitioningAndOrdering.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanPartitioningAndOrdering.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.sql.execution.datasources.v2 -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.CLASS_NAME import org.apache.spark.sql.catalyst.expressions.V2ExpressionUtils import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala index 078acb72ce072..f9bb4d62b32d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala @@ -22,7 +22,6 @@ import java.util.Locale import scala.collection.mutable import org.apache.spark.internal.LogKeys.{AGGREGATE_FUNCTIONS, COLUMN_NAMES, GROUP_BY_EXPRS, JOIN_CONDITION, JOIN_TYPE, POST_SCAN_FILTERS, PUSHED_FILTERS, RELATION_NAME, RELATION_OUTPUT} -import org.apache.spark.internal.MDC import org.apache.spark.sql.catalyst.expressions.{aggregate, Alias, And, Attribute, AttributeMap, AttributeReference, AttributeSet, Cast, Expression, IntegerLiteral, Literal, NamedExpression, PredicateHelper, ProjectionOverSchema, SortOrder, SubqueryExpression} import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.optimizer.CollapseProject diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala index 5801f0580de26..68e8c1c6e1047 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala @@ -23,7 +23,7 @@ import java.util import scala.jdk.CollectionConverters._ import org.apache.spark.{SparkEnv, SparkException, TaskContext} -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.{InternalRow, ProjectingInternalRow} import org.apache.spark.sql.catalyst.analysis.NoSuchTableException diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala index e7cb0f57460a3..dd1ce5fe9fd23 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc import scala.util.control.NonFatal -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{JOIN_CONDITION, JOIN_TYPE, SCHEMA} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.connector.expressions.{FieldReference, SortOrder} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonStreamingPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonStreamingPartitionReaderFactory.scala index 7d80cc2728102..466ecf6090938 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonStreamingPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonStreamingPartitionReaderFactory.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources.v2.python import org.apache.spark.SparkEnv -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.connector.metric.CustomTaskMetric import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/metadata/StateMetadataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/metadata/StateMetadataSource.scala index c5ccff9228d6e..ae4483154e25b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/metadata/StateMetadataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/metadata/StateMetadataSource.scala @@ -23,7 +23,7 @@ import scala.jdk.CollectionConverters._ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{Path, PathFilter} -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericInternalRow diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala index 7f69657988a45..8c695f4f3958d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala @@ -26,7 +26,6 @@ import scala.util.control.NonFatal import org.apache.spark.{broadcast, SparkException} import org.apache.spark.internal.LogKeys._ -import org.apache.spark.internal.MDC import org.apache.spark.rdd.{RDD, RDDOperationScope} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeRow diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index de5c3aaa4fe4d..568db3cf5e5b0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.exchange import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import org.apache.spark.internal.{LogKeys, MDC} +import org.apache.spark.internal.{LogKeys} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/AttachDistributedSequenceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/AttachDistributedSequenceExec.scala index a8a6fa97c52a9..e27bde38a6f5f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/AttachDistributedSequenceExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/AttachDistributedSequenceExec.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.python import org.apache.spark.internal.LogKeys.{RDD_ID, SPARK_PLAN_ID} -import org.apache.spark.internal.MDC import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala index bede14db9cbad..1407e020353f0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala @@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.SparkException import org.apache.spark.api.python.PythonEvalType -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.REASON import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonPlannerRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonPlannerRunner.scala index 7834b591e2e80..200a92f3dc3c6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonPlannerRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonPlannerRunner.scala @@ -30,7 +30,7 @@ import net.razorvine.pickle.Pickler import org.apache.spark.{JobArtifactSet, SparkEnv, SparkException} import org.apache.spark.api.python.{BasePythonRunner, PythonFunction, PythonWorker, PythonWorkerUtils, SpecialLengths} -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.config.BUFFER_SIZE import org.apache.spark.internal.config.Python._ import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/ApplyInPandasWithStatePythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/ApplyInPandasWithStatePythonRunner.scala index 6f2c1a986c279..1ba219837f43f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/ApplyInPandasWithStatePythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/ApplyInPandasWithStatePythonRunner.scala @@ -28,7 +28,6 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.api.python._ import org.apache.spark.internal.LogKeys.CONFIG -import org.apache.spark.internal.MDC import org.apache.spark.sql.Row import org.apache.spark.sql.api.python.PythonSQLUtils import org.apache.spark.sql.catalyst.InternalRow diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/PythonStreamingSourceRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/PythonStreamingSourceRunner.scala index 3979220618baa..5945b91084278 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/PythonStreamingSourceRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/PythonStreamingSourceRunner.scala @@ -28,7 +28,7 @@ import org.apache.arrow.vector.ipc.ArrowStreamReader import org.apache.spark.SparkEnv import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths} -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.LogKeys.PYTHON_EXEC import org.apache.spark.internal.config.BUFFER_SIZE import org.apache.spark.internal.config.Python.PYTHON_AUTH_SOCKET_TIMEOUT diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkStateServer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkStateServer.scala index 5c8a55edaffc1..4b50cee44e737 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkStateServer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkStateServer.scala @@ -29,7 +29,7 @@ import org.apache.arrow.vector.VectorSchemaRoot import org.apache.arrow.vector.ipc.ArrowStreamWriter import org.apache.spark.SparkEnv -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.config.Python.PYTHON_UNIX_DOMAIN_SOCKET_ENABLED import org.apache.spark.sql.{Encoders, Row} import org.apache.spark.sql.api.python.PythonSQLUtils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/ArrowRRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/ArrowRRunner.scala index aaf2f256273d3..fafe1fbe313ab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/ArrowRRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/ArrowRRunner.scala @@ -30,7 +30,7 @@ import org.apache.spark.TaskContext import org.apache.spark.api.r._ import org.apache.spark.api.r.SpecialLengths import org.apache.spark.broadcast.Broadcast -import org.apache.spark.internal.{LogKeys, MDC} +import org.apache.spark.internal.{LogKeys} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.arrow.ArrowWriter import org.apache.spark.sql.types.StructType diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/AsyncCommitLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/AsyncCommitLog.scala index 6db01624fd26b..dce2ea9f14cee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/AsyncCommitLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/AsyncCommitLog.scala @@ -22,7 +22,7 @@ import java.util.concurrent.{CompletableFuture, ConcurrentLinkedDeque, ThreadPoo import scala.jdk.CollectionConverters._ -import org.apache.spark.internal.{LogKeys, MDC} +import org.apache.spark.internal.{LogKeys} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.errors.QueryExecutionErrors diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/AsyncOffsetSeqLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/AsyncOffsetSeqLog.scala index 54a8855b77cdb..cd31ae4baa317 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/AsyncOffsetSeqLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/AsyncOffsetSeqLog.scala @@ -23,7 +23,7 @@ import java.util.concurrent.atomic.AtomicLong import scala.jdk.CollectionConverters._ -import org.apache.spark.internal.{LogKeys, MDC} +import org.apache.spark.internal.{LogKeys} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.util.{Clock, SystemClock} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/CheckpointFileManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/CheckpointFileManager.scala index 793215fb24562..afc4a901348d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/CheckpointFileManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/CheckpointFileManager.scala @@ -27,7 +27,7 @@ import org.apache.hadoop.fs._ import org.apache.hadoop.fs.local.{LocalFs, RawLocalFs} import org.apache.hadoop.fs.permission.FsPermission -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{FINAL_PATH, PATH, TEMP_PATH} import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.streaming.CheckpointFileManager.RenameHelperMethods diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/HDFSMetadataLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/HDFSMetadataLog.scala index ee06087827f8b..62b9c665e70b9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/HDFSMetadataLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/HDFSMetadataLog.scala @@ -28,7 +28,7 @@ import org.apache.hadoop.fs._ import org.json4s.{Formats, NoTypeHints} import org.json4s.jackson.Serialization -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.errors.QueryExecutionErrors diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/OffsetSeq.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/OffsetSeq.scala index a599f3bc66118..24d56672ccad7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/OffsetSeq.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/OffsetSeq.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.streaming import org.json4s.{Formats, NoTypeHints} import org.json4s.jackson.Serialization -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CONFIG, DEFAULT_VALUE, NEW_VALUE, OLD_VALUE, TIP} import org.apache.spark.io.CompressionCodec import org.apache.spark.sql.RuntimeConfig diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala index a13c00ee20576..43381fe034af4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala @@ -26,7 +26,6 @@ import scala.collection.mutable.{Map => MutableMap} import org.apache.spark.SparkEnv import org.apache.spark.internal.LogKeys._ -import org.apache.spark.internal.MDC import org.apache.spark.sql.catalyst.expressions.{CurrentDate, CurrentTimestampLike, LocalTimestamp} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.streaming.{StreamingRelationV2, WriteToStream} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousQueuedDataReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousQueuedDataReader.scala index 398df496d15ff..d9eff62bcc986 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousQueuedDataReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousQueuedDataReader.scala @@ -23,7 +23,7 @@ import java.util.concurrent.{ArrayBlockingQueue, TimeUnit} import scala.util.control.NonFatal import org.apache.spark.{SparkEnv, TaskContext} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeProjection diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala index 273ffa6aefb7b..8adde9d3ff15b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala @@ -29,7 +29,7 @@ import org.json4s.{DefaultFormats, Formats, NoTypeHints} import org.json4s.jackson.Serialization import org.apache.spark.SparkEnv -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{HOST, PORT} import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.sql.Encoders diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousWriteRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousWriteRDD.scala index d5daa9a875f83..51399969df1b9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousWriteRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousWriteRDD.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.streaming.continuous import org.apache.spark.{Partition, SparkEnv, TaskContext} -import org.apache.spark.internal.{LogKeys, MDC} +import org.apache.spark.internal.{LogKeys} import org.apache.spark.internal.LogKeys._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala index 42ce32e1bc674..8b5d6c580f105 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.streaming.continuous -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/StreamingSessionWindowStateManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/StreamingSessionWindowStateManager.scala index 71df9dc65b419..8220045eaf212 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/StreamingSessionWindowStateManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/StreamingSessionWindowStateManager.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.streaming.state -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, Literal, UnsafeProjection, UnsafeRow} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/join/SymmetricHashJoinStateManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/join/SymmetricHashJoinStateManager.scala index 619671d99e57e..50e508da4b7db 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/join/SymmetricHashJoinStateManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/join/SymmetricHashJoinStateManager.scala @@ -24,7 +24,7 @@ import scala.annotation.tailrec import org.apache.hadoop.conf.Configuration import org.apache.spark.TaskContext -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{END_INDEX, START_INDEX, STATE_STORE_ID} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, JoinedRow, Literal, SafeProjection, SpecificInternalRow, UnsafeProjection, UnsafeRow} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/timers/TimerStateImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/timers/TimerStateImpl.scala index ede45fc924a98..6f6a9997b3ba9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/timers/TimerStateImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/timers/TimerStateImpl.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.sql.execution.streaming -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{EXPIRY_TIMESTAMP, KEY} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AsyncProgressTrackingMicroBatchExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AsyncProgressTrackingMicroBatchExecution.scala index bbfedd5454deb..df3f1fdd8c903 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AsyncProgressTrackingMicroBatchExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AsyncProgressTrackingMicroBatchExecution.scala @@ -21,7 +21,6 @@ import java.util.concurrent._ import java.util.concurrent.atomic.AtomicLong import org.apache.spark.internal.LogKeys.{BATCH_ID, PRETTY_ID_STRING} -import org.apache.spark.internal.MDC import org.apache.spark.sql.catalyst.streaming.WriteToStream import org.apache.spark.sql.classic.SparkSession import org.apache.spark.sql.errors.QueryExecutionErrors diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AvailableNowDataStreamWrapper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AvailableNowDataStreamWrapper.scala index f42250c3c702d..f70c255c28edb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AvailableNowDataStreamWrapper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AvailableNowDataStreamWrapper.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.streaming -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{DELEGATE, READ_LIMIT} import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, ReadLimit, SparkDataStream, SupportsAdmissionControl, SupportsTriggerAvailableNow} import org.apache.spark.sql.connector.read.streaming diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/CompactibleFileStreamLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/CompactibleFileStreamLog.scala index d6770452e71f3..d76f3152286f2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/CompactibleFileStreamLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/CompactibleFileStreamLog.scala @@ -27,7 +27,7 @@ import org.apache.hadoop.fs.Path import org.json4s.{Formats, NoTypeHints} import org.json4s.jackson.Serialization -import org.apache.spark.internal.{LogKeys, MDC} +import org.apache.spark.internal.{LogKeys} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.util.Utils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamSource.scala index 465973cabe587..801d42a1f6b2f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamSource.scala @@ -27,7 +27,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, GlobFilter, Path} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.paths.SparkPath import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.classic.{DataFrame, Dataset, SparkSession} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/IncrementalExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/IncrementalExecution.scala index b6701182d7e06..f602de39ca9d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/IncrementalExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/IncrementalExecution.scala @@ -24,7 +24,7 @@ import scala.collection.mutable.{Map => MutableMap} import org.apache.hadoop.fs.Path -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{BATCH_TIMESTAMP, ERROR} import org.apache.spark.sql.catalyst.QueryPlanningTracker import org.apache.spark.sql.catalyst.expressions.{CurrentBatchTimestamp, ExpressionWithRandomSeed} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ManifestFileCommitProtocol.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ManifestFileCommitProtocol.scala index 6574dfd9b5bd0..a628126990305 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ManifestFileCommitProtocol.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ManifestFileCommitProtocol.scala @@ -25,7 +25,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{BATCH_ID, PATH} import org.apache.spark.internal.io.{FileCommitProtocol, FileNameSpec} import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MetadataLogFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MetadataLogFileIndex.scala index 45bb69a9c056b..16ca548fabb6b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MetadataLogFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MetadataLogFileIndex.scala @@ -22,7 +22,6 @@ import scala.collection.mutable import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.spark.internal.LogKeys._ -import org.apache.spark.internal.MDC import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.types.StructType diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MicroBatchExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MicroBatchExecution.scala index 45712cf087c46..7a9f5db8c5119 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MicroBatchExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MicroBatchExecution.scala @@ -21,7 +21,7 @@ import scala.collection.mutable.{Map => MutableMap} import scala.collection.mutable import scala.util.control.NonFatal -import org.apache.spark.internal.{LogKeys, MDC} +import org.apache.spark.internal.{LogKeys} import org.apache.spark.internal.LogKeys.BATCH_ID import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, CurrentBatchTimestamp, CurrentDate, CurrentTimestamp, FileSourceMetadataAttribute, LocalTimestamp} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ProgressReporter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ProgressReporter.scala index 8f07126a33bbb..a64ac31608cba 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ProgressReporter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ProgressReporter.scala @@ -26,7 +26,7 @@ import java.util.concurrent.atomic.AtomicBoolean import scala.collection.mutable import scala.jdk.CollectionConverters._ -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.optimizer.InlineCTE import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, LogicalPlan, WithCTE} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ResolveWriteToStream.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ResolveWriteToStream.scala index 6c0a228ec2fca..a41abeb0a66b1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ResolveWriteToStream.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ResolveWriteToStream.scala @@ -24,7 +24,6 @@ import scala.util.control.NonFatal import org.apache.hadoop.fs.Path import org.apache.spark.internal.LogKeys.{CHECKPOINT_LOCATION, CHECKPOINT_ROOT, CONFIG, PATH} -import org.apache.spark.internal.MDC import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamExecution.scala index 8dd055949289a..4fd124d1831fd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamExecution.scala @@ -33,7 +33,7 @@ import org.apache.hadoop.fs.Path import org.apache.logging.log4j.CloseableThreadContext import org.apache.spark.{JobArtifactSet, SparkContext, SparkException, SparkThrowable} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CHECKPOINT_PATH, CHECKPOINT_ROOT, LOGICAL_PLAN, PATH, PRETTY_ID_STRING, QUERY_ID, RUN_ID, SPARK_DATA_STREAM} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamMetadata.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamMetadata.scala index 1c6ecabaa8edb..19949d719e2ad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamMetadata.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamMetadata.scala @@ -27,7 +27,7 @@ import org.apache.hadoop.fs.{FileAlreadyExistsException, FSDataInputStream, Path import org.json4s.{Formats, NoTypeHints} import org.json4s.jackson.Serialization -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.streaming.CheckpointFileManager.CancellableFSDataOutputStream import org.apache.spark.util.Utils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/TriggerExecutor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/TriggerExecutor.scala index bfa838e43e288..5457387aa3f2e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/TriggerExecutor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/TriggerExecutor.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.streaming -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{ELAPSED_TIME, TRIGGER_INTERVAL} import org.apache.spark.util.{Clock, SystemClock} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/WatermarkTracker.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/WatermarkTracker.scala index 7228767c4d18a..55eb823eadb68 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/WatermarkTracker.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/WatermarkTracker.scala @@ -21,7 +21,7 @@ import java.util.{Locale, UUID} import scala.collection.mutable -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.RuntimeConfig import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, LogicalPlan} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sinks/FileStreamSink.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sinks/FileStreamSink.scala index 11e764821c71a..83c7ced1b6d80 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sinks/FileStreamSink.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sinks/FileStreamSink.scala @@ -23,7 +23,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.SparkException -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{BATCH_ID, ERROR, PATH} import org.apache.spark.internal.io.FileCommitProtocol import org.apache.spark.sql.{DataFrame, SparkSession} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sinks/FileStreamSinkLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sinks/FileStreamSinkLog.scala index 556438811c44d..82fbc49de439c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sinks/FileStreamSinkLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sinks/FileStreamSinkLog.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.execution.streaming import org.apache.hadoop.fs.FileStatus import org.apache.spark.internal.LogKeys._ -import org.apache.spark.internal.MDC import org.apache.spark.paths.SparkPath import org.apache.spark.sql.SparkSession import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchStream.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchStream.scala index f3319280f2602..b166b92c3b898 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchStream.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchStream.scala @@ -21,7 +21,7 @@ import java.io._ import java.nio.charset.StandardCharsets import java.util.concurrent.TimeUnit -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketMicroBatchStream.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketMicroBatchStream.scala index 597b981ebe556..ed93c6dda7a6c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketMicroBatchStream.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketMicroBatchStream.scala @@ -25,7 +25,7 @@ import javax.annotation.concurrent.GuardedBy import scala.collection.mutable.ListBuffer -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{HOST, PORT} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.DateTimeUtils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala index 6cfaadaa69951..c362ac916384f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala @@ -31,7 +31,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ import org.apache.spark.{SparkConf, SparkEnv, SparkException, TaskContext} -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.io.CompressionCodec import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.errors.QueryExecutionErrors diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadata.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadata.scala index 1c97e95847908..ee3b3cdb6d9eb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadata.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadata.scala @@ -28,7 +28,7 @@ import org.json4s.{Formats, JBool, JObject, NoTypeHints} import org.json4s.jackson.JsonMethods.{compact, render} import org.json4s.jackson.Serialization -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceErrors import org.apache.spark.sql.execution.streaming.{CheckpointFileManager, CommitLog, MetadataVersionUtil, StateStoreWriter, StreamingQueryCheckpointMetadata} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala index b47b2db3f9cb1..7367ad7485bda 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala @@ -37,7 +37,7 @@ import org.rocksdb.CompressionType._ import org.rocksdb.TickerType._ import org.apache.spark.TaskContext -import org.apache.spark.internal.{LogEntry, Logging, LogKeys, MDC} +import org.apache.spark.internal.{LogEntry, Logging, LogKeys} import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.util.{NextIterator, Utils} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala index 9fa13a1cfc645..d3e3b52d9c24f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala @@ -37,7 +37,7 @@ import org.json4s.{Formats, NoTypeHints} import org.json4s.jackson.Serialization import org.apache.spark.{SparkConf, SparkEnv, SparkException} -import org.apache.spark.internal.{Logging, LogKeys, MDC, MessageWithContext} +import org.apache.spark.internal.{Logging, LogKeys, MessageWithContext} import org.apache.spark.internal.LogKeys.{DFS_FILE, VERSION_NUM} import org.apache.spark.io.CompressionCodec import org.apache.spark.sql.errors.QueryExecutionErrors diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBMemoryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBMemoryManager.scala index 5b0c7d6eacdd4..c92dbf69720af 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBMemoryManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBMemoryManager.scala @@ -24,7 +24,7 @@ import scala.jdk.CollectionConverters._ import org.rocksdb._ import org.apache.spark.SparkEnv -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.memory.{MemoryMode, UnifiedMemoryManager, UnmanagedMemoryConsumer, UnmanagedMemoryConsumerId} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateMachine.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateMachine.scala index 0ef45645bc6c7..80ef42f9bd75b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateMachine.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateMachine.scala @@ -24,7 +24,7 @@ import javax.annotation.concurrent.GuardedBy import scala.ref.WeakReference import org.apache.spark.TaskContext -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.sql.errors.QueryExecutionErrors /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala index 0cf32244fc654..36480691a516b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala @@ -27,7 +27,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.{SparkConf, SparkEnv, TaskContext} -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.LogKeys._ import org.apache.spark.io.CompressionCodec import org.apache.spark.sql.catalyst.expressions.UnsafeRow diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala index 08cfd8fb197a3..143263075101c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FSDataInputStream, Path} import org.apache.spark.SparkUnsupportedOperationException -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.avro.{AvroDeserializer, AvroSerializer, SchemaConverters} import org.apache.spark.sql.catalyst.util.UnsafeRowUtils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala index a9d3c75776e09..af0e7069eeef8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala @@ -33,7 +33,7 @@ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods.{compact, render} import org.apache.spark.{SparkContext, SparkEnv, SparkException, TaskContext} -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.LogKeys.{EXCEPTION, STATE_STORE_ID, VERSION_NUM} import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.catalyst.util.UnsafeRowUtils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala index 94f0274087390..ccb58ed05a6db 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.fs.{FSError, Path} import org.json4s._ import org.json4s.jackson.Serialization -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.io.CompressionCodec import org.apache.spark.sql.catalyst.expressions.UnsafeRow diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinator.scala index f280553b95406..85292657c11ff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinator.scala @@ -22,7 +22,7 @@ import java.util.UUID import scala.collection.mutable import org.apache.spark.SparkEnv -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala index f680860231f01..4dbee24f0fecd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala @@ -25,7 +25,7 @@ import scala.jdk.CollectionConverters._ import scala.util.control.NonFatal import org.apache.spark.{JobExecutionStatus, SparkConf} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.CLASS_NAME import org.apache.spark.internal.config.Status._ import org.apache.spark.scheduler._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala index 2b1451493398f..635e183f9a025 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -29,7 +29,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FsUrlStreamHandlerFactory, Path} import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CONFIG, CONFIG2, PATH, VALUE} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.errors.QueryExecutionErrors diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala index c043f90f45980..d7d730ed0cc3a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala @@ -27,7 +27,6 @@ import scala.util.control.NonFatal import org.apache.spark.SparkThrowable import org.apache.spark.internal.LogKeys.COLUMN_NAME -import org.apache.spark.internal.MDC import org.apache.spark.sql.catalyst.SQLConfHelper import org.apache.spark.sql.catalyst.analysis.{IndexAlreadyExistsException, NonEmptyNamespaceException, NoSuchIndexException} import org.apache.spark.sql.connector.catalog.Identifier diff --git a/sql/core/src/test/scala/org/apache/spark/sql/LogQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/LogQuerySuite.scala index 861b0bf0f3945..854b50a65d37d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/LogQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/LogQuerySuite.scala @@ -18,8 +18,9 @@ package org.apache.spark.sql import java.io.File +import java.util.Locale -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.LogUtils.SPARK_LOG_SCHEMA @@ -63,7 +64,8 @@ class LogQuerySuite extends QueryTest with SharedSparkSession with Logging { createTempView("logs") checkAnswer( spark.sql(s"SELECT level, msg, context, exception FROM logs WHERE msg = '${msg.message}'"), - Row("ERROR", msg.message, Map(LogKeys.EXECUTOR_ID.name -> "1"), null) :: Nil) + Row("ERROR", msg.message, + Map(LogKeys.EXECUTOR_ID.name.toLowerCase(Locale.ROOT) -> "1"), null) :: Nil) } } @@ -74,7 +76,7 @@ class LogQuerySuite extends QueryTest with SharedSparkSession with Logging { withTempView("logs") { createTempView("logs") - val expectedMDC = Map(LogKeys.TASK_ID.name -> "2") + val expectedMDC = Map(LogKeys.TASK_ID.name.toLowerCase(Locale.ROOT) -> "2") checkAnswer( spark.sql("SELECT level, msg, context, exception.class, exception.msg FROM logs " + s"WHERE msg = '${msg.message}'"), diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/AbstractService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/AbstractService.java index b31d024eeeeb9..1062f63971880 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/AbstractService.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/AbstractService.java @@ -88,7 +88,7 @@ public synchronized void init(HiveConf hiveConf) { ensureCurrentState(STATE.NOTINITED); this.hiveConf = hiveConf; changeState(STATE.INITED); - LOG.info("Service:{} is inited.", MDC.of(LogKeys.SERVICE_NAME$.MODULE$, getName())); + LOG.info("Service:{} is inited.", MDC.of(LogKeys.SERVICE_NAME, getName())); } /** @@ -103,7 +103,7 @@ public synchronized void start() { startTime = System.currentTimeMillis(); ensureCurrentState(STATE.INITED); changeState(STATE.STARTED); - LOG.info("Service:{} is started.", MDC.of(LogKeys.SERVICE_NAME$.MODULE$, getName())); + LOG.info("Service:{} is started.", MDC.of(LogKeys.SERVICE_NAME, getName())); } /** @@ -124,7 +124,7 @@ public synchronized void stop() { } ensureCurrentState(STATE.STARTED); changeState(STATE.STOPPED); - LOG.info("Service:{} is stopped.", MDC.of(LogKeys.SERVICE_NAME$.MODULE$, getName())); + LOG.info("Service:{} is stopped.", MDC.of(LogKeys.SERVICE_NAME, getName())); } @Override diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/CompositeService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/CompositeService.java index 663bcdb86f9f6..5b65c134a3a0d 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/CompositeService.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/CompositeService.java @@ -73,7 +73,7 @@ public synchronized void start() { } super.start(); } catch (Throwable e) { - LOG.error("Error starting services {}", e, MDC.of(LogKeys.SERVICE_NAME$.MODULE$, getName())); + LOG.error("Error starting services {}", e, MDC.of(LogKeys.SERVICE_NAME, getName())); // Note that the state of the failed service is still INITED and not // STARTED. Even though the last service is not started completely, still // call stop() on all services including failed service to make sure cleanup @@ -103,7 +103,7 @@ private synchronized void stop(int numOfServicesStarted) { try { service.stop(); } catch (Throwable t) { - LOG.info("Error stopping {}", t, MDC.of(LogKeys.SERVICE_NAME$.MODULE$, service.getName())); + LOG.info("Error stopping {}", t, MDC.of(LogKeys.SERVICE_NAME, service.getName())); } } } @@ -127,7 +127,7 @@ public void run() { compositeService.stop(); } catch (Throwable t) { LOG.info("Error stopping {}", t, - MDC.of(LogKeys.SERVICE_NAME$.MODULE$, compositeService.getName())); + MDC.of(LogKeys.SERVICE_NAME, compositeService.getName())); } } } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceOperations.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceOperations.java index 92d733c563cab..f945eed0a5c60 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceOperations.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceOperations.java @@ -133,7 +133,7 @@ public static Exception stopQuietly(Service service) { stop(service); } catch (Exception e) { LOG.warn("When stopping the service {}", e, - MDC.of(LogKeys.SERVICE_NAME$.MODULE$, service.getName())); + MDC.of(LogKeys.SERVICE_NAME, service.getName())); return e; } return null; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java index 2bd6210f58c76..da4c170f93fd7 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java @@ -289,7 +289,7 @@ public String verifyDelegationToken(String delegationToken) throws HiveSQLExcept return delegationTokenManager.verifyDelegationToken(delegationToken); } catch (IOException e) { String msg = "Error verifying delegation token"; - LOG.error(msg + " {}", e, MDC.of(LogKeys.TOKEN$.MODULE$, delegationToken)); + LOG.error(msg + " {}", e, MDC.of(LogKeys.TOKEN, delegationToken)); throw new HiveSQLException(msg + delegationToken, "08S01", e); } } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java index e307bdab04498..a76bcc9b873e5 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java @@ -113,7 +113,7 @@ public static String getUserNameFromCookieToken(String tokenStr) { if (!map.keySet().equals(COOKIE_ATTRIBUTES)) { LOG.error("Invalid token with missing attributes {}", - MDC.of(LogKeys.TOKEN$.MODULE$, tokenStr)); + MDC.of(LogKeys.TOKEN, tokenStr)); return null; } return map.get(COOKIE_CLIENT_USER_NAME); @@ -133,7 +133,7 @@ private static Map splitCookieToken(String tokenStr) { String part = st.nextToken(); int separator = part.indexOf(COOKIE_KEY_VALUE_SEPARATOR); if (separator == -1) { - LOG.error("Invalid token string {}", MDC.of(LogKeys.TOKEN$.MODULE$, tokenStr)); + LOG.error("Invalid token string {}", MDC.of(LogKeys.TOKEN, tokenStr)); return null; } String key = part.substring(0, separator); diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIService.java index 86fb725d3a3cc..0ab721bbd2a1f 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIService.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIService.java @@ -103,8 +103,8 @@ public synchronized void init(HiveConf hiveConf) { String keyTabFile = hiveConf.getVar(ConfVars.HIVE_SERVER2_SPNEGO_KEYTAB); if (principal.isEmpty() || keyTabFile.isEmpty()) { LOG.info("SPNego httpUGI not created, spNegoPrincipal: {}, keytabFile: {}", - MDC.of(LogKeys.PRINCIPAL$.MODULE$, principal), - MDC.of(LogKeys.KEYTAB_FILE$.MODULE$, keyTabFile)); + MDC.of(LogKeys.PRINCIPAL, principal), + MDC.of(LogKeys.KEYTAB_FILE, keyTabFile)); } else { try { this.httpUGI = HiveAuthFactory.loginFromSpnegoKeytabAndReturnUGI(hiveConf); @@ -462,7 +462,7 @@ public OperationStatus getOperationStatus(OperationHandle opHandle) } catch (ExecutionException e) { // The background operation thread was aborted LOG.warn("{}: The background operation was aborted", e, - MDC.of(LogKeys.OPERATION_HANDLE$.MODULE$, opHandle)); + MDC.of(LogKeys.OPERATION_HANDLE, opHandle)); } catch (InterruptedException e) { // No op, this thread was interrupted // In this case, the call might return sooner than long polling timeout @@ -556,7 +556,7 @@ public String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory au String owner, String renewer) throws HiveSQLException { String delegationToken = sessionManager.getSession(sessionHandle) .getDelegationToken(authFactory, owner, renewer); - LOG.info("{}: getDelegationToken()", MDC.of(LogKeys.SESSION_HANDLE$.MODULE$, sessionHandle)); + LOG.info("{}: getDelegationToken()", MDC.of(LogKeys.SESSION_HANDLE, sessionHandle)); return delegationToken; } @@ -564,14 +564,14 @@ public String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory au public void cancelDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, String tokenStr) throws HiveSQLException { sessionManager.getSession(sessionHandle).cancelDelegationToken(authFactory, tokenStr); - LOG.info("{}: cancelDelegationToken()", MDC.of(LogKeys.SESSION_HANDLE$.MODULE$, sessionHandle)); + LOG.info("{}: cancelDelegationToken()", MDC.of(LogKeys.SESSION_HANDLE, sessionHandle)); } @Override public void renewDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, String tokenStr) throws HiveSQLException { sessionManager.getSession(sessionHandle).renewDelegationToken(authFactory, tokenStr); - LOG.info("{}: renewDelegationToken()", MDC.of(LogKeys.SESSION_HANDLE$.MODULE$, sessionHandle)); + LOG.info("{}: renewDelegationToken()", MDC.of(LogKeys.SESSION_HANDLE, sessionHandle)); } @Override diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java index 4331f6829fbf3..e573d3d43738c 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java @@ -71,7 +71,7 @@ public ColumnBasedSet(TRowSet tRowSet) throws TException { try { tvalue.read(protocol); } catch (TException e) { - LOG.error("{}", e, MDC.of(LogKeys.ERROR$.MODULE$, e.getMessage())); + LOG.error("{}", e, MDC.of(LogKeys.ERROR, e.getMessage())); throw new TException("Error reading column value from the row set blob", e); } columns.add(new ColumnBuffer(tvalue)); diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java index 0b71b606b9d65..026a66b0355f9 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java @@ -73,7 +73,7 @@ public String[] mapToHiveType(String clientTypeName) { Collection hiveTableType = clientToHiveMap.get(clientTypeName.toUpperCase()); if (hiveTableType == null) { LOG.warn("Not supported client table type {}", - MDC.of(LogKeys.TABLE_TYPE$.MODULE$, clientTypeName)); + MDC.of(LogKeys.TABLE_TYPE, clientTypeName)); return new String[] {clientTypeName}; } return Iterables.toArray(hiveTableType, String.class); @@ -84,7 +84,7 @@ public String mapToClientType(String hiveTypeName) { String clientTypeName = hiveToClientMap.get(hiveTypeName); if (clientTypeName == null) { LOG.warn("Invalid hive table type {}", - MDC.of(LogKeys.TABLE_TYPE$.MODULE$, hiveTypeName)); + MDC.of(LogKeys.TABLE_TYPE, hiveTypeName)); return hiveTypeName; } return clientTypeName; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java index f488a411c31f3..b5cf8991a4cb3 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java @@ -212,7 +212,7 @@ protected void createOperationLog() { try { if (operationLogFile.exists()) { LOG.warn("The operation log file should not exist, but it is already there: {}", - MDC.of(LogKeys.PATH$.MODULE$, operationLogFile.getAbsolutePath())); + MDC.of(LogKeys.PATH, operationLogFile.getAbsolutePath())); operationLogFile.delete(); } if (!operationLogFile.createNewFile()) { @@ -221,14 +221,14 @@ protected void createOperationLog() { if (!operationLogFile.canRead() || !operationLogFile.canWrite()) { LOG.warn("The already existed operation log file cannot be recreated, " + "and it cannot be read or written: {}", - MDC.of(LogKeys.PATH$.MODULE$, operationLogFile.getAbsolutePath())); + MDC.of(LogKeys.PATH, operationLogFile.getAbsolutePath())); isOperationLogEnabled = false; return; } } } catch (Exception e) { LOG.warn("Unable to create operation log file: {}", e, - MDC.of(LogKeys.PATH$.MODULE$, operationLogFile.getAbsolutePath())); + MDC.of(LogKeys.PATH, operationLogFile.getAbsolutePath())); isOperationLogEnabled = false; return; } @@ -238,7 +238,7 @@ protected void createOperationLog() { operationLog = new OperationLog(opHandle.toString(), operationLogFile, parentSession.getHiveConf()); } catch (FileNotFoundException e) { LOG.warn("Unable to instantiate OperationLog object for operation: {}", e, - MDC.of(LogKeys.OPERATION_HANDLE$.MODULE$, opHandle)); + MDC.of(LogKeys.OPERATION_HANDLE, opHandle)); isOperationLogEnabled = false; return; } @@ -290,7 +290,7 @@ protected void cleanupOperationLog() { if (operationLog == null) { LOG.error("Operation [ {} ] logging is enabled, " + "but its OperationLog object cannot be found.", - MDC.of(LogKeys.OPERATION_HANDLE_ID$.MODULE$, opHandle.getHandleIdentifier())); + MDC.of(LogKeys.OPERATION_HANDLE_ID, opHandle.getHandleIdentifier())); } else { operationLog.close(); } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java index fd8266d1a9acc..79c12d387a9dd 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java @@ -293,7 +293,7 @@ public List removeExpiredOperations(OperationHandle[] handles) { Operation operation = removeTimedOutOperation(handle); if (operation != null) { LOG.warn("Operation {} is timed-out and will be closed", - MDC.of(LogKeys.OPERATION_HANDLE$.MODULE$, handle)); + MDC.of(LogKeys.OPERATION_HANDLE, handle)); removed.add(operation); } } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java index 8ec7a64db8ec0..19aa7ab385ae2 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java @@ -152,7 +152,7 @@ public void open(Map sessionConfMap) throws HiveSQLException { sessionState.loadReloadableAuxJars(); } catch (IOException e) { String msg = "Failed to load reloadable jar file path."; - LOG.error("{}", e, MDC.of(LogKeys.ERROR$.MODULE$, msg)); + LOG.error("{}", e, MDC.of(LogKeys.ERROR, msg)); throw new HiveSQLException(msg, e); } // Process global init file: .hiverc @@ -203,7 +203,7 @@ private void processGlobalInitFile() { } if (hivercFile.isFile()) { LOG.info("Running global init file: {}", - MDC.of(LogKeys.GLOBAL_INIT_FILE$.MODULE$, hivercFile)); + MDC.of(LogKeys.GLOBAL_INIT_FILE, hivercFile)); int rc = processor.processFile(hivercFile.getAbsolutePath()); if (rc != 0) { LOG.error("Failed on initializing global .hiverc file"); @@ -304,28 +304,28 @@ private static void setConf(String varname, String key, String varvalue, boolean public void setOperationLogSessionDir(File operationLogRootDir) { if (!operationLogRootDir.exists()) { LOG.warn("The operation log root directory is removed, recreating: {}", - MDC.of(LogKeys.PATH$.MODULE$, operationLogRootDir.getAbsolutePath())); + MDC.of(LogKeys.PATH, operationLogRootDir.getAbsolutePath())); if (!Utils.createDirectory(operationLogRootDir)) { LOG.warn("Unable to create operation log root directory: {}", - MDC.of(LogKeys.PATH$.MODULE$, operationLogRootDir.getAbsolutePath())); + MDC.of(LogKeys.PATH, operationLogRootDir.getAbsolutePath())); } } if (!operationLogRootDir.canWrite()) { LOG.warn("The operation log root directory is not writable: {}", - MDC.of(LogKeys.PATH$.MODULE$, operationLogRootDir.getAbsolutePath())); + MDC.of(LogKeys.PATH, operationLogRootDir.getAbsolutePath())); } sessionLogDir = new File(operationLogRootDir, sessionHandle.getHandleIdentifier().toString()); isOperationLogEnabled = true; if (!sessionLogDir.exists()) { if (!sessionLogDir.mkdir()) { LOG.warn("Unable to create operation log session directory: {}", - MDC.of(LogKeys.PATH$.MODULE$, sessionLogDir.getAbsolutePath())); + MDC.of(LogKeys.PATH, sessionLogDir.getAbsolutePath())); isOperationLogEnabled = false; } } if (isOperationLogEnabled) { LOG.info("Operation log session directory is created: {}", - MDC.of(LogKeys.PATH$.MODULE$, sessionLogDir.getAbsolutePath())); + MDC.of(LogKeys.PATH, sessionLogDir.getAbsolutePath())); } } @@ -661,7 +661,7 @@ public void close() throws HiveSQLException { operationManager.closeOperation(opHandle); } catch (Exception e) { LOG.warn("Exception is thrown closing operation {}", e, - MDC.of(LogKeys.OPERATION_HANDLE$.MODULE$, opHandle)); + MDC.of(LogKeys.OPERATION_HANDLE, opHandle)); } } opHandleSet.clear(); @@ -710,13 +710,13 @@ private void cleanupPipeoutFile() { if (fileAry == null) { LOG.error("Unable to access pipeout files in {}", - MDC.of(LogKeys.LOCAL_SCRATCH_DIR$.MODULE$, lScratchDir)); + MDC.of(LogKeys.LOCAL_SCRATCH_DIR, lScratchDir)); } else { for (File file : fileAry) { try { JavaUtils.deleteRecursively(file); } catch (Exception e) { - LOG.error("Failed to cleanup pipeout file: {}", e, MDC.of(LogKeys.PATH$.MODULE$, file)); + LOG.error("Failed to cleanup pipeout file: {}", e, MDC.of(LogKeys.PATH, file)); } } } @@ -728,7 +728,7 @@ private void cleanupSessionLogDir() { JavaUtils.deleteRecursively(sessionLogDir); } catch (Exception e) { LOG.error("Failed to cleanup session log dir: {}", e, - MDC.of(LogKeys.SESSION_HANDLE$.MODULE$, sessionHandle)); + MDC.of(LogKeys.SESSION_HANDLE, sessionHandle)); } } } @@ -778,7 +778,7 @@ private void closeTimedOutOperations(List operations) { operation.close(); } catch (Exception e) { LOG.warn("Exception is thrown closing timed-out operation {}", e, - MDC.of(LogKeys.OPERATION_HANDLE$.MODULE$, operation.getHandle())); + MDC.of(LogKeys.OPERATION_HANDLE, operation.getHandle())); } } } finally { diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java index 27c98683d85e5..9e7a86b0747cc 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java @@ -89,14 +89,14 @@ public synchronized void init(HiveConf hiveConf) { private void createBackgroundOperationPool() { int poolSize = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_ASYNC_EXEC_THREADS); LOG.info("HiveServer2: Background operation thread pool size: {}", - MDC.of(LogKeys.THREAD_POOL_SIZE$.MODULE$, poolSize)); + MDC.of(LogKeys.THREAD_POOL_SIZE, poolSize)); int poolQueueSize = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_ASYNC_EXEC_WAIT_QUEUE_SIZE); LOG.info("HiveServer2: Background operation thread wait queue size: {}", - MDC.of(LogKeys.THREAD_POOL_WAIT_QUEUE_SIZE$.MODULE$, poolQueueSize)); + MDC.of(LogKeys.THREAD_POOL_WAIT_QUEUE_SIZE, poolQueueSize)); long keepAliveTime = HiveConf.getTimeVar( hiveConf, ConfVars.HIVE_SERVER2_ASYNC_EXEC_KEEPALIVE_TIME, TimeUnit.SECONDS); LOG.info("HiveServer2: Background operation thread keepalive time: {} ms", - MDC.of(LogKeys.THREAD_POOL_KEEPALIVE_TIME$.MODULE$, keepAliveTime * 1000L)); + MDC.of(LogKeys.THREAD_POOL_KEEPALIVE_TIME, keepAliveTime * 1000L)); // Create a thread pool with #poolSize threads // Threads terminate when they are idle for more than the keepAliveTime @@ -122,26 +122,26 @@ private void initOperationLogRootDir() { if (operationLogRootDir.exists() && !operationLogRootDir.isDirectory()) { LOG.warn("The operation log root directory exists, but it is not a directory: {}", - MDC.of(LogKeys.PATH$.MODULE$, operationLogRootDir.getAbsolutePath())); + MDC.of(LogKeys.PATH, operationLogRootDir.getAbsolutePath())); isOperationLogEnabled = false; } if (!operationLogRootDir.exists()) { if (!Utils.createDirectory(operationLogRootDir)) { LOG.warn("Unable to create operation log root directory: {}", - MDC.of(LogKeys.PATH$.MODULE$, operationLogRootDir.getAbsolutePath())); + MDC.of(LogKeys.PATH, operationLogRootDir.getAbsolutePath())); isOperationLogEnabled = false; } } if (isOperationLogEnabled) { LOG.info("Operation log root directory is created: {}", - MDC.of(LogKeys.PATH$.MODULE$, operationLogRootDir.getAbsolutePath())); + MDC.of(LogKeys.PATH, operationLogRootDir.getAbsolutePath())); try { FileUtils.forceDeleteOnExit(operationLogRootDir); } catch (IOException e) { LOG.warn("Failed to schedule cleanup HS2 operation logging root dir: {}", e, - MDC.of(LogKeys.PATH$.MODULE$, operationLogRootDir.getAbsolutePath())); + MDC.of(LogKeys.PATH, operationLogRootDir.getAbsolutePath())); } } } @@ -172,13 +172,13 @@ public void run() { && (!checkOperation || session.getNoOperationTime() > sessionTimeout)) { SessionHandle handle = session.getSessionHandle(); LOG.warn("Session {} is Timed-out (last access : {}) and will be closed", - MDC.of(LogKeys.SESSION_HANDLE$.MODULE$, handle), - MDC.of(LogKeys.LAST_ACCESS_TIME$.MODULE$, new Date(session.getLastAccessTime()))); + MDC.of(LogKeys.SESSION_HANDLE, handle), + MDC.of(LogKeys.LAST_ACCESS_TIME, new Date(session.getLastAccessTime()))); try { closeSession(handle); } catch (HiveSQLException e) { LOG.warn("Exception is thrown closing session {}", e, - MDC.of(LogKeys.SESSION_HANDLE$.MODULE$, handle)); + MDC.of(LogKeys.SESSION_HANDLE, handle)); } } else { session.closeExpiredOperations(); @@ -221,7 +221,7 @@ public synchronized void stop() { } catch (InterruptedException e) { LOG.warn("HIVE_SERVER2_ASYNC_EXEC_SHUTDOWN_TIMEOUT = {} ms has been exceeded. " + "RUNNING background operations will be shut down", e, - MDC.of(LogKeys.TIMEOUT$.MODULE$, timeout * 1000)); + MDC.of(LogKeys.TIMEOUT, timeout * 1000)); } backgroundOperationPool = null; } @@ -234,7 +234,7 @@ private void cleanupLoggingRootDir() { Utils.deleteRecursively(operationLogRootDir); } catch (Exception e) { LOG.warn("Failed to cleanup root dir of HS2 logging: {}", e, - MDC.of(LogKeys.PATH$.MODULE$, operationLogRootDir.getAbsolutePath())); + MDC.of(LogKeys.PATH, operationLogRootDir.getAbsolutePath())); } } } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index 920c002433c5b..80d7b4e369d4c 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -251,7 +251,7 @@ private TStatus notSupportTokenErrorStatus() { @Override public TOpenSessionResp OpenSession(TOpenSessionReq req) throws TException { LOG.info("Client protocol version: {}", - MDC.of(LogKeys.PROTOCOL_VERSION$.MODULE$, req.getClient_protocol())); + MDC.of(LogKeys.PROTOCOL_VERSION, req.getClient_protocol())); TOpenSessionResp resp = new TOpenSessionResp(); try { SessionHandle sessionHandle = getSessionHandle(req, resp); @@ -287,7 +287,7 @@ public TSetClientInfoResp SetClientInfo(TSetClientInfoReq req) throws TException sb.append(e.getKey()).append(" = ").append(e.getValue()); } if (sb != null) { - LOG.info("{}", MDC.of(LogKeys.SET_CLIENT_INFO_REQUEST$.MODULE$, sb)); + LOG.info("{}", MDC.of(LogKeys.SET_CLIENT_INFO_REQUEST, sb)); } } return new TSetClientInfoResp(OK_STATUS); diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java index d9bf361fdef63..dabce675649e2 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java @@ -178,7 +178,7 @@ protected void doPost(HttpServletRequest request, HttpServletResponse response) response.addCookie(hs2Cookie); } LOG.info("Cookie added for clientUserName {}", - MDC.of(LogKeys.USER_NAME$.MODULE$, clientUserName)); + MDC.of(LogKeys.USER_NAME, clientUserName)); } super.doPost(request, response); } @@ -232,7 +232,7 @@ private String getClientNameFromCookie(Cookie[] cookies) { String userName = HttpAuthUtils.getUserNameFromCookieToken(currValue); if (userName == null) { - LOG.warn("Invalid cookie token {}", MDC.of(LogKeys.TOKEN$.MODULE$, currValue)); + LOG.warn("Invalid cookie token {}", MDC.of(LogKeys.TOKEN, currValue)); continue; } //We have found a valid cookie in the client request. diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java index 46ee775e8dd49..9e3ec3fc61ce8 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java @@ -145,7 +145,7 @@ private static void startHiveServer2() throws Throwable { throw new Error("Max start attempts " + maxAttempts + " exhausted", throwable); } else { LOG.warn("Error starting HiveServer2 on attempt {}, will retry in 60 seconds", - throwable, MDC.of(LogKeys.NUM_RETRY$.MODULE$, attempts)); + throwable, MDC.of(LogKeys.NUM_RETRY, attempts)); try { Thread.sleep(60L * 1000L); } catch (InterruptedException e) { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index b952e953b1425..350aba1ab175b 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -30,7 +30,7 @@ import org.apache.hive.service.cli.operation.ExecuteStatementOperation import org.apache.hive.service.cli.session.HiveSession import org.apache.hive.service.rpc.thrift.{TCLIServiceConstants, TColumnDesc, TPrimitiveTypeEntry, TRowSet, TTableSchema, TTypeDesc, TTypeEntry, TTypeId, TTypeQualifiers, TTypeQualifierValue} -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.{DataFrame, Row, SparkSession} import org.apache.spark.sql.catalyst.util.CharVarcharUtils diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala index 8dfe551892fad..a02b2ca8966de 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala @@ -22,7 +22,7 @@ import org.apache.hive.service.cli.OperationState import org.apache.hive.service.cli.operation.GetCatalogsOperation import org.apache.hive.service.cli.session.HiveSession -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.SparkSession diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala index 4560856cb0634..6c573ceb14ecf 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala @@ -27,7 +27,7 @@ import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.GetColumnsOperation import org.apache.hive.service.cli.session.HiveSession -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.TableIdentifier diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala index c59875b90c449..ec0da495d8744 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala @@ -27,7 +27,7 @@ import org.apache.hive.service.cli.operation.GetFunctionsOperation import org.apache.hive.service.cli.operation.MetadataOperation.DEFAULT_HIVE_CATALOG import org.apache.hive.service.cli.session.HiveSession -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.sql.SparkSession /** diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala index 1db286a7a7f29..4b8b603eede59 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala @@ -25,7 +25,7 @@ import org.apache.hive.service.cli.operation.GetSchemasOperation import org.apache.hive.service.cli.operation.MetadataOperation.DEFAULT_HIVE_CATALOG import org.apache.hive.service.cli.session.HiveSession -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.SparkSession diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala index f8ed09857f1c9..e3e1cdbd36813 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala @@ -24,7 +24,7 @@ import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.GetTableTypesOperation import org.apache.hive.service.cli.session.HiveSession -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.catalog.CatalogTableType diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala index d57c590156d07..0579d567d0228 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala @@ -27,7 +27,7 @@ import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.GetTablesOperation import org.apache.hive.service.cli.session.HiveSession -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.catalog.CatalogTableType._ diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala index c982eaaef6394..4af2265841b59 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala @@ -26,7 +26,7 @@ import org.apache.hive.service.cli.OperationState import org.apache.hive.service.cli.operation.GetTypeInfoOperation import org.apache.hive.service.cli.session.HiveSession -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.SparkSession diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala index f653e899ebf49..fea878bcf05d5 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala @@ -21,7 +21,7 @@ import org.apache.hive.service.cli.{HiveSQLException, OperationState} import org.apache.hive.service.cli.operation.Operation import org.apache.spark.SparkContext -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{HIVE_OPERATION_TYPE, STATEMENT_ID} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.CurrentUserContext.CURRENT_USER diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index c7e1bc5412ee1..02826abef8571 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -39,7 +39,7 @@ import sun.misc.{Signal, SignalHandler} import org.apache.spark.{ErrorMessageFormat, SparkConf, SparkThrowable, SparkThrowableHelper} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.FunctionRegistry diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala index 7cc181ea6945a..3b0fd9267d311 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala @@ -27,7 +27,7 @@ import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse import org.apache.spark.SparkThrowable -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.COMMAND import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.plans.logical.CommandResult diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala index 8d03d5f848b76..ee77776a77f2c 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala @@ -25,7 +25,7 @@ import scala.jdk.CollectionConverters._ import org.apache.hive.service.server.HiveServer2 import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config.Status.LIVE_ENTITY_UPDATE_PERIOD import org.apache.spark.scheduler._ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 5d4cdd7c78e7f..6389e130b6a2c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -34,7 +34,7 @@ import org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_FORMAT import org.apache.thrift.TException import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{DATABASE_NAME, INCOMPATIBLE_TYPES, PROVIDER, SCHEMA, SCHEMA2, TABLE_NAME} import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.TableIdentifier diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index bd35342b909f3..fcad044e58715 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -25,7 +25,7 @@ import com.google.common.util.concurrent.Striped import org.apache.hadoop.fs.Path import org.apache.spark.SparkException -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{QualifiedTableName, TableIdentifier} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala index 253ee81b31b64..a36501d1eaf41 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala @@ -36,7 +36,7 @@ import org.apache.hive.common.util.HiveVersionInfo import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.classic.SQLContext diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index 8af0403b1d9b2..d6f62284d2e7c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -36,7 +36,7 @@ import org.apache.hadoop.mapred.{FileInputFormat, InputFormat => oldInputClass, import org.apache.hadoop.mapreduce.{InputFormat => newInputClass} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, NewHadoopRDD, RDD, UnionRDD} import org.apache.spark.sql.SparkSession diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 9389bb425492e..3383ff8421951 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -50,7 +50,7 @@ import org.apache.thrift.transport.{TEndpointTransport, TTransport} import org.apache.spark.{SparkConf, SparkException, SparkThrowable} import org.apache.spark.deploy.SparkHadoopUtil.SOURCE_SPARK -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.metrics.source.HiveCatalogMetrics diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index 24c5cfb72c7ec..ef27669f5ba09 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -38,7 +38,7 @@ import org.apache.hadoop.hive.ql.processors.{CommandProcessor, CommandProcessorF import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde.serdeConstants -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CONFIG, CONFIG2, CONFIG3} import org.apache.spark.metrics.source.HiveCatalogMetrics import org.apache.spark.sql.catalyst.{FunctionIdentifier, InternalRow} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala index 1adc2f623b762..0bfe588df72c3 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala @@ -29,7 +29,7 @@ import org.apache.hadoop.hive.shims.ShimLoader import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkSubmit -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{FALLBACK_VERSION, HADOOP_VERSION, PATH} import org.apache.spark.sql.catalyst.util.quietly import org.apache.spark.sql.errors.QueryExecutionErrors diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala index d8f9405780d8c..33bf6ae1554cb 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala @@ -31,7 +31,7 @@ import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred.{JobConf, Reporter} import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.CLASS_NAME import org.apache.spark.internal.config.SPECULATION_ENABLED import org.apache.spark.sql.SparkSession diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTempPath.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTempPath.scala index d97d3cd6dd4a9..3864d1601504c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTempPath.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTempPath.scala @@ -31,7 +31,7 @@ import org.apache.hadoop.hive.common.FileUtils import org.apache.hadoop.hive.ql.exec.TaskRunner import org.apache.spark.SparkException -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.PATH import org.apache.spark.sql.SparkSession import org.apache.spark.sql.errors.QueryExecutionErrors diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala index d588e9f5bd5c4..f0730b743fe67 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.hive.ql.io.orc.{OrcFile, Reader} import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.PATH import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.errors.QueryExecutionErrors diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/security/HiveDelegationTokenProvider.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/security/HiveDelegationTokenProvider.scala index 9113b9be8f3c3..d7f4305018609 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/security/HiveDelegationTokenProvider.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/security/HiveDelegationTokenProvider.scala @@ -32,7 +32,7 @@ import org.apache.hadoop.security.token.Token import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.CLASS_NAME import org.apache.spark.internal.config.KEYTAB import org.apache.spark.security.HadoopDelegationTokenProvider diff --git a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/DatasetManager.scala b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/DatasetManager.scala index 0d393e307aaa7..05e12a3f7859f 100644 --- a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/DatasetManager.scala +++ b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/DatasetManager.scala @@ -21,7 +21,7 @@ import scala.jdk.CollectionConverters._ import scala.util.control.{NonFatal, NoStackTrace} import org.apache.spark.SparkException -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.connector.catalog.{ diff --git a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/FlowExecution.scala b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/FlowExecution.scala index 5c981a2442edd..68daa60c8b1a3 100644 --- a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/FlowExecution.scala +++ b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/FlowExecution.scala @@ -24,7 +24,7 @@ import scala.concurrent.{ExecutionContext, Future} import scala.util.{Failure, Success} import scala.util.control.NonFatal -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.classic.SparkSession import org.apache.spark.sql.pipelines.graph.QueryOrigin.ExceptionHelpers diff --git a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/GraphExecution.scala b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/GraphExecution.scala index 381449711dbbb..4c969f1bbefd0 100644 --- a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/GraphExecution.scala +++ b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/GraphExecution.scala @@ -22,7 +22,7 @@ import scala.concurrent.ExecutionContext import scala.jdk.CollectionConverters._ import scala.util.{Failure, Success} -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.pipelines.logging.StreamListener diff --git a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/TriggeredGraphExecution.scala b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/TriggeredGraphExecution.scala index 503a1aa8e2816..c57f125aed331 100644 --- a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/TriggeredGraphExecution.scala +++ b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/TriggeredGraphExecution.scala @@ -25,7 +25,7 @@ import scala.jdk.CollectionConverters._ import scala.util.Try import scala.util.control.NonFatal -import org.apache.spark.internal.{LogKeys, MDC} +import org.apache.spark.internal.{LogKeys} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.pipelines.graph.TriggeredGraphExecution._ import org.apache.spark.sql.pipelines.util.ExponentialBackoffStrategy diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index f09f9caf129bd..1a4e6a2a402c0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -28,7 +28,7 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.LogKeys.{BACKUP_FILE, CHECKPOINT_FILE, CHECKPOINT_TIME, NUM_RETRY, PATH, TEMP_FILE} import org.apache.spark.internal.config.UI._ import org.apache.spark.io.CompressionCodec diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala index 52e0379c2a48d..c286c33866549 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala @@ -22,7 +22,7 @@ import java.io.{IOException, ObjectInputStream, ObjectOutputStream} import scala.collection.mutable import scala.collection.parallel.immutable.ParVector -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.streaming.dstream.{DStream, InputDStream, ReceiverInputDStream} import org.apache.spark.streaming.scheduler.Job import org.apache.spark.util.Utils diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 94b695e6452e5..79bc38318f919 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -36,7 +36,7 @@ import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.input.FixedLengthBinaryInputFormat -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.rdd.{RDD, RDDOperationScope} import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.SerializationDebugger diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index f0f9046c6b623..74edbd9996dd4 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -26,7 +26,7 @@ import scala.reflect.ClassTag import scala.util.matching.Regex import org.apache.spark.{SparkContext, SparkException} -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.LogKeys.{FROM_TIME, SLIDE_DURATION, TO_TIME} import org.apache.spark.internal.io.SparkHadoopWriterUtils import org.apache.spark.rdd.{BlockRDD, RDD, RDDOperationScope} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala index 128a5fded49a9..7845db610ac92 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala @@ -24,7 +24,7 @@ import scala.reflect.ClassTag import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.LogKeys.{PATH, TIME} import org.apache.spark.streaming.Time import org.apache.spark.util.Utils diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala index b067c505da0dd..f7cdd047d9ebf 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} -import org.apache.spark.internal.{LogKeys, MDC} +import org.apache.spark.internal.{LogKeys} import org.apache.spark.internal.LogKeys._ import org.apache.spark.rdd.{RDD, UnionRDD} import org.apache.spark.streaming._ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala index 2deb388eb4b82..2cb10e4513437 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala @@ -23,7 +23,6 @@ import scala.reflect.ClassTag import org.apache.spark.SparkContext import org.apache.spark.internal.LogKeys.{LAST_VALID_TIME, TIME} -import org.apache.spark.internal.MDC import org.apache.spark.rdd.RDDOperationScope import org.apache.spark.streaming.{Duration, StreamingContext, Time} import org.apache.spark.streaming.scheduler.RateController diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala index de7882032122a..8466e9f70843d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala @@ -25,7 +25,7 @@ import java.util.concurrent.ArrayBlockingQueue import scala.reflect.ClassTag -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.receiver.Receiver diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala index 83f36b760db3a..256e065b5cac3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala @@ -24,7 +24,7 @@ import java.nio.charset.StandardCharsets import scala.reflect.ClassTag import scala.util.control.NonFatal -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.receiver.Receiver diff --git a/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala index 12c6c95f7d8d3..4b48288822752 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala @@ -24,7 +24,7 @@ import scala.reflect.ClassTag import scala.util.control.NonFatal import org.apache.spark._ -import org.apache.spark.internal.{LogKeys, MDC} +import org.apache.spark.internal.{LogKeys} import org.apache.spark.rdd.BlockRDD import org.apache.spark.storage.{BlockId, StorageLevel} import org.apache.spark.streaming.util._ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala index fae68123773dd..47c579b0bfce7 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala @@ -22,7 +22,7 @@ import java.util.concurrent.{ArrayBlockingQueue, TimeUnit} import scala.collection.mutable.ArrayBuffer import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.LogKeys._ import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.StreamingConf.BLOCK_INTERVAL diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala index e513a75b69903..85fe8b857727c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala @@ -24,7 +24,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{EFFECTIVE_STORAGE_LEVEL, STORAGE_LEVEL, STORAGE_LEVEL_DESERIALIZED, STORAGE_LEVEL_REPLICATION} import org.apache.spark.serializer.SerializerManager import org.apache.spark.storage._ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala index 7cc08b421f780..0850922578304 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala @@ -25,7 +25,7 @@ import scala.concurrent._ import scala.util.control.NonFatal import org.apache.spark.SparkConf -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.LogKeys.{DELAY, ERROR, MESSAGE, STREAM_ID} import org.apache.spark.storage.StreamBlockId import org.apache.spark.util.{ThreadUtils, Utils} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index aafa99bd5285d..6a997ede2b7e3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -28,7 +28,7 @@ import com.google.common.base.Throwables import org.apache.hadoop.conf.Configuration import org.apache.spark.{SparkEnv, SparkException} -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.LogKeys.{ERROR, MESSAGE} import org.apache.spark.rpc.{RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.storage.StreamBlockId diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManager.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManager.scala index 903cde8082db7..e55914e17dbae 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManager.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManager.scala @@ -21,7 +21,7 @@ package org.apache.spark.streaming.scheduler import scala.util.Random import org.apache.spark.{ExecutorAllocationClient, SparkConf} -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.config.DECOMMISSION_ENABLED import org.apache.spark.internal.config.Streaming._ import org.apache.spark.resource.ResourceProfile diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/InputInfoTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/InputInfoTracker.scala index bd9ea7b5a2688..0021c2536e804 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/InputInfoTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/InputInfoTracker.scala @@ -20,7 +20,7 @@ package org.apache.spark.streaming.scheduler import scala.collection.mutable import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.streaming.{StreamingContext, Time} /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index 856695ac2c982..2e82e7db74ff6 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -21,7 +21,7 @@ import java.util.concurrent.TimeUnit import scala.util.{Failure, Success, Try} -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.LogKeys._ import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Checkpoint, CheckpointWriter, StreamingConf, Time} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index 852b5cd492a25..79c041cd5c109 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -24,7 +24,7 @@ import scala.jdk.CollectionConverters._ import scala.util.Failure import org.apache.spark.ExecutorAllocationClient -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.io.SparkHadoopWriterUtils import org.apache.spark.rdd.RDD import org.apache.spark.streaming._ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala index 7fb35a04be6da..b76208f873761 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala @@ -27,7 +27,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.SparkConf -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.LogKeys.{RECEIVED_BLOCK_INFO, RECEIVED_BLOCK_TRACKER_LOG_EVENT} import org.apache.spark.network.util.JavaUtils import org.apache.spark.streaming.Time diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index a37ba04c10123..c2ca04c7be931 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -24,7 +24,7 @@ import scala.concurrent.ExecutionContext import scala.util.{Failure, Success} import org.apache.spark._ -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.LogKeys.{ERROR, MESSAGE, RECEIVER_ID, RECEIVER_IDS, STREAM_ID} import org.apache.spark.rdd.RDD import org.apache.spark.rpc._ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimator.scala index 1b05a6ac30cc4..ad6e7a0b68952 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimator.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.scheduler.rate -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} /** * Implements a proportional-integral-derivative (PID) controller which acts on diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala index 8befe53efffa7..de7aaced39fac 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala @@ -29,7 +29,7 @@ import scala.jdk.CollectionConverters._ import scala.util.control.NonFatal import org.apache.spark.SparkConf -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.LogKeys.RECORDS import org.apache.spark.network.util.JavaUtils import org.apache.spark.util.{ThreadUtils, Utils} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala index d90095c73785a..024de96b986e1 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala @@ -31,7 +31,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.SparkConf -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.LogKeys.{NUM_RETRY, WRITE_AHEAD_LOG_INFO} import org.apache.spark.util.{CompletionIterator, ThreadUtils} import org.apache.spark.util.ArrayImplicits._ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala index 459338f3517fa..0a98c87e90008 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala @@ -24,7 +24,7 @@ import java.nio.ByteBuffer import scala.io.Source import org.apache.spark.SparkConf -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.{IntParam, Utils} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala index 80363d626bbad..ad50a71e061c3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.util -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.util.{Clock, SystemClock} private[streaming] diff --git a/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala index 793730ef0b3ac..44bf852b95ac3 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala @@ -32,7 +32,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.scalatest.Assertions._ -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.PATH import org.apache.spark.streaming.dstream.DStream import org.apache.spark.util.Utils