From 8f747e3dead852619d5ec5fe0c677c1814ee9bbd Mon Sep 17 00:00:00 2001 From: Amanda Liu Date: Thu, 13 Jun 2024 11:31:09 -0700 Subject: [PATCH 01/18] [DRAFT] Scala Structured Logging Migration --- .../org/apache/spark/internal/LogKey.scala | 29 ++ .../org/apache/spark/util/MavenUtils.scala | 9 +- .../streaming/StreamingQueryListenerBus.scala | 7 +- ...ConnectStreamingQueryListenerHandler.scala | 23 +- .../apache/spark/broadcast/Broadcast.scala | 3 +- .../spark/deploy/ExternalShuffleService.scala | 4 +- .../spark/deploy/FaultToleranceTest.scala | 453 ------------------ .../spark/deploy/worker/WorkerWatcher.scala | 4 +- .../shuffle/IndexShuffleBlockResolver.scala | 4 +- ...ableStreamBasedCheckpointFileManager.scala | 4 +- .../classification/LogisticRegression.scala | 12 +- .../spark/deploy/yarn/ApplicationMaster.scala | 13 +- .../org/apache/spark/deploy/yarn/Client.scala | 7 +- .../cluster/YarnClientSchedulerBackend.scala | 2 +- scalastyle-config.xml | 8 + .../spark/sql/types/UDTRegistration.scala | 5 +- .../expressions/codegen/CodeGenerator.scala | 15 +- .../adaptive/ShufflePartitionsUtil.scala | 6 +- ...ythonStreamingPartitionReaderFactory.scala | 5 +- .../python/PythonStreamingSourceRunner.scala | 3 +- .../streaming/ProgressReporter.scala | 9 +- .../execution/streaming/StreamExecution.scala | 8 +- .../continuous/ContinuousExecution.scala | 3 +- .../continuous/ContinuousWriteRDD.scala | 8 +- .../streaming/state/StateStoreChangelog.scala | 8 +- .../SparkExecuteStatementOperation.scala | 16 +- .../SparkGetFunctionsOperation.scala | 4 +- .../thriftserver/SparkSQLCLIService.scala | 8 +- .../apache/spark/streaming/Checkpoint.scala | 38 +- .../spark/streaming/StreamingContext.scala | 3 +- .../spark/streaming/dstream/DStream.scala | 39 +- .../dstream/SocketInputDStream.scala | 8 +- .../receiver/ReceiverSupervisor.scala | 9 +- .../receiver/ReceiverSupervisorImpl.scala | 6 +- .../scheduler/ExecutorAllocationManager.scala | 24 +- .../scheduler/InputInfoTracker.scala | 5 +- .../scheduler/ReceivedBlockTracker.scala | 17 +- .../streaming/scheduler/ReceiverTracker.scala | 19 +- .../scheduler/rate/PIDRateEstimator.scala | 9 +- .../streaming/util/BatchedWriteAheadLog.scala | 5 +- .../util/FileBasedWriteAheadLog.scala | 19 +- 41 files changed, 271 insertions(+), 610 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala 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 index b8b63382fe4c..2ed1ac6d66ec 100644 --- a/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala +++ b/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala @@ -66,6 +66,8 @@ object LogKeys { case object ADMIN_ACLS extends LogKey case object ADMIN_ACL_GROUPS 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 APP_ATTEMPT_ID extends LogKey @@ -77,8 +79,10 @@ object LogKeys { case object APP_STATE extends LogKey case object ARCHIVE_NAME extends LogKey case object ARGS 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 @@ -98,6 +102,8 @@ object LogKeys { case object BROADCAST_ID extends LogKey case object BROADCAST_OUTPUT_STATUS_SIZE extends LogKey case object BUCKET extends LogKey + case object BYTE_BUFFER extends LogKey + case object BYTE_SIZE extends LogKey case object BYTECODE_SIZE extends LogKey case object CACHED_TABLE_PARTITION_METADATA_SIZE extends LogKey case object CACHE_AUTO_REMOVED_SIZE extends LogKey @@ -108,7 +114,9 @@ object LogKeys { case object CANCEL_FUTURE_JOBS extends LogKey case object CATALOG_NAME extends LogKey case object CATEGORICAL_FEATURES extends LogKey + case object CHECKPOINT_DIR 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 @@ -186,17 +194,20 @@ object LogKeys { 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 DIRECTORY extends LogKey case object DIVISIBLE_CLUSTER_INDICES_SIZE extends LogKey case object DRIVER_ID 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 @@ -242,6 +253,7 @@ object LogKeys { case object EXPR extends LogKey case object EXPR_TERMS extends LogKey case object EXTENDED_EXPLAIN_GENERATOR extends LogKey + case object FAILED_ARTIFACTS extends LogKey case object FAILED_STAGE extends LogKey case object FAILED_STAGE_NAME extends LogKey case object FAILURES extends LogKey @@ -251,6 +263,7 @@ object LogKeys { case object FEATURE_NAME extends LogKey case object FETCH_SIZE extends LogKey case object FIELD_NAME 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 @@ -307,6 +320,7 @@ object LogKeys { case object INIT_MODE extends LogKey case object INPUT extends LogKey case object INPUT_SPLIT extends LogKey + case object INTEGRAL extends LogKey case object INTERVAL extends LogKey case object ISOLATION_LEVEL extends LogKey case object ISSUE_DATE extends LogKey @@ -347,6 +361,8 @@ object LogKeys { case object LOCATION extends LogKey case object LOGICAL_PLAN_COLUMNS extends LogKey case object LOGICAL_PLAN_LEAVES extends LogKey + case object LOG_DIRECTORY extends LogKey + case object LOG_FILES extends LogKey case object LOG_ID extends LogKey case object LOG_KEY_FILE extends LogKey case object LOG_LEVEL extends LogKey @@ -394,6 +410,7 @@ object LogKeys { 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_SHARE extends LogKey case object MIN_SIZE extends LogKey case object MIN_TIME extends LogKey @@ -490,6 +507,7 @@ object LogKeys { 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 @@ -575,6 +593,7 @@ object LogKeys { 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 @@ -595,6 +614,7 @@ object LogKeys { case object QUERY_PLAN_LENGTH_MAX extends LogKey case object QUERY_RUN_ID extends LogKey case object RANGE extends LogKey + case object RATIO extends LogKey case object RDD_CHECKPOINT_DIR extends LogKey case object RDD_DEBUG_STRING extends LogKey case object RDD_DESCRIPTION extends LogKey @@ -646,10 +666,14 @@ object LogKeys { case object RULE_NAME extends LogKey case object RUN_ID extends LogKey case object SCALA_VERSION extends LogKey + case object SCALING_UP_RATIO extends LogKey + case object SCALING_DOWN_RATIO extends LogKey + case object SCALING_INTERVAL_SECS 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 SENDER_ADDRESS extends LogKey case object SERVER_NAME extends LogKey case object SERVICE_NAME extends LogKey case object SERVLET_CONTEXT_HANDLER_PATH extends LogKey @@ -677,6 +701,7 @@ object LogKeys { case object SMALLEST_CLUSTER_INDEX 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 @@ -751,6 +776,7 @@ object LogKeys { 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 TID extends LogKey case object TIME extends LogKey case object TIMEOUT extends LogKey @@ -792,6 +818,7 @@ object LogKeys { case object URL extends LogKey case object URL2 extends LogKey case object URLS extends LogKey + case object USER_CLASS extends LogKey case object USER_ID extends LogKey case object USER_NAME extends LogKey case object UUID extends LogKey @@ -801,6 +828,7 @@ object LogKeys { case object VOCAB_SIZE extends LogKey case object WAIT_RESULT_TIME extends LogKey case object WAIT_SEND_TIME extends LogKey + case object WAIT_TIME extends LogKey case object WATERMARK_CONSTRAINT extends LogKey case object WEB_URL extends LogKey case object WEIGHT extends LogKey @@ -814,4 +842,5 @@ object LogKeys { case object XML_SCHEDULING_MODE extends LogKey case object XSD_PATH 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/util/MavenUtils.scala b/common/utils/src/main/scala/org/apache/spark/util/MavenUtils.scala index ae00987cd69f..641af47e85ae 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 @@ -36,7 +36,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 +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.util.ArrayImplicits._ /** Provides utility functions to be used inside SparkSubmit. */ @@ -215,7 +215,7 @@ private[spark] object MavenUtils extends Logging { if (artifactInfo.getExt == "jar") { true } else { - logInfo(s"Skipping non-jar dependency ${artifactInfo.getId}") + logInfo(log"Skipping non-jar dependency ${MDC(LogKeys.ARTIFACT_ID, artifactInfo.getId)}") false } } @@ -515,8 +515,9 @@ private[spark] object MavenUtils extends Logging { val failedReports = rr.getArtifactsReports(DownloadStatus.FAILED, true) if (failedReports.nonEmpty && noCacheIvySettings.isDefined) { val failedArtifacts = failedReports.map(r => r.getArtifact) - logInfo(s"Download failed: ${failedArtifacts.mkString("[", ", ", "]")}, " + - s"attempt to retry while skipping local-m2-cache.") + logInfo(log"Download failed: ${MDC(LogKeys.FAILED_ARTIFACTS, + failedArtifacts.mkString("[", ", ", "]"))}, " + + log"attempt to retry while skipping local-m2-cache.") failedArtifacts.foreach(artifact => { clearInvalidIvyCacheFiles(artifact.getModuleRevisionId, ivySettings.getDefaultCache) }) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListenerBus.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListenerBus.scala index 56a9e19a1b78..a3391c7c9fbc 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListenerBus.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/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 +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.connect.client.CloseableIterator import org.apache.spark.sql.streaming.StreamingQueryListener.{Event, QueryIdleEvent, QueryProgressEvent, QueryStartedEvent, QueryTerminatedEvent} @@ -115,7 +115,7 @@ class StreamingQueryListenerBus(sparkSession: SparkSession) extends Logging { case StreamingQueryEventType.QUERY_TERMINATED_EVENT => postToAll(QueryTerminatedEvent.fromJson(event.getEventJson)) case _ => - logWarning(s"Unknown StreamingQueryListener event: $event") + logWarning(log"Unknown StreamingQueryListener event: ${MDC(LogKeys.EVENT, event)}") } }) } @@ -144,7 +144,8 @@ class StreamingQueryListenerBus(sparkSession: SparkSession) extends Logging { listener.onQueryIdle(t) case t: QueryTerminatedEvent => listener.onQueryTerminated(t) - case _ => logWarning(s"Unknown StreamingQueryListener event: $event") + case _ => logWarning(log"Unknown StreamingQueryListener event: " + + log"${MDC(LogKeys.EVENT, event)}") } } catch { case e: Exception => diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala index 94f01026b7a5..de4bdf3d4451 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala +++ b/connector/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 +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.sql.connect.service.ExecuteHolder /** @@ -83,20 +83,27 @@ class SparkConnectStreamingQueryListenerHandler(executeHolder: ExecuteHolder) ex } catch { case NonFatal(e) => logError( - s"[SessionId: $sessionId][UserId: $userId][operationId: " + - s"${executeHolder.operationId}] Error sending listener added response.", + log"[SessionId: ${MDC(LogKeys.SESSION_ID, sessionId)}]" + + log"[UserId: ${MDC(LogKeys.USER_ID, userId)}]" + + log"[operationId: ${MDC(LogKeys.OPERATION_HANDLE_IDENTIFIER, + executeHolder.operationId)}] " + + log"Error sending listener added response.", e) listenerHolder.cleanUp() return } } - logInfo(s"[SessionId: $sessionId][UserId: $userId][operationId: " + - s"${executeHolder.operationId}] Server side listener added. Now blocking until " + - "all client side listeners are removed or there is error transmitting the event back.") + logInfo(log"[SessionId: ${MDC(LogKeys.SESSION_ID, sessionId)}][UserId: " + + log"${MDC(LogKeys.USER_ID, userId)}][operationId: " + + log"${MDC(LogKeys.OPERATION_HANDLE_IDENTIFIER, executeHolder.operationId)}] " + + log"Server side listener added. Now blocking until all client side listeners are " + + log"removed or there is error transmitting the event back.") // Block the handling thread, and have serverListener continuously send back new events listenerHolder.streamingQueryListenerLatch.await() - logInfo(s"[SessionId: $sessionId][UserId: $userId][operationId: " + - s"${executeHolder.operationId}] Server side listener long-running handling thread ended.") + logInfo(log"[SessionId: ${MDC(LogKeys.SESSION_ID, sessionId)}][UserId: " + + log"${MDC(LogKeys.USER_ID, userId)}]" + + log"[operationId: ${MDC(LogKeys.OPERATION_HANDLE, executeHolder.operationId)}] " + + log"Server side listener long-running handling thread ended.") case StreamingQueryListenerBusCommand.CommandCase.REMOVE_LISTENER_BUS_LISTENER => listenerHolder.isServerSideListenerRegistered match { case true => 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 445b7d4d7aa0..672bef88c1cc 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala @@ -106,7 +106,8 @@ abstract class Broadcast[T: ClassTag](val id: Long) extends Serializable with Lo assertValid() _isValid = false _destroySite = Utils.getCallSite().shortForm - logInfo("Destroying %s (from %s)".format(toString, _destroySite)) + logInfo(log"Destroying ${MDC(LogKeys.TO_STRING, toString)} " + + log"(from ${MDC(LogKeys.DESTROY_SITE, _destroySite)})") doDestroy(blocking) } 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 3ce5e2d62b6a..f7ec118d7160 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala @@ -71,8 +71,8 @@ class ExternalShuffleService(sparkConf: SparkConf, securityManager: SecurityMana if (localDirs.length >= 1) { new File(localDirs.find(new File(_, dbName).exists()).getOrElse(localDirs(0)), dbName) } else { - logWarning(s"'spark.local.dir' should be set first when we use db in " + - s"ExternalShuffleService. Note that this only affects standalone mode.") + logWarning(log"'spark.local.dir' should be set first when we use db in " + + log"ExternalShuffleService. Note that this only affects standalone mode.") null } } diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala deleted file mode 100644 index d6a50ff84f56..000000000000 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ /dev/null @@ -1,453 +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.deploy - -import java.io._ -import java.net.URL -import java.nio.charset.StandardCharsets -import java.util.concurrent.TimeoutException - -import scala.collection.mutable.ListBuffer -import scala.concurrent.{Future, Promise} -// scalastyle:off executioncontextglobal -import scala.concurrent.ExecutionContext.Implicits.global -// scalastyle:on executioncontextglobal -import scala.concurrent.duration._ -import scala.sys.process._ - -import org.json4s.Formats -import org.json4s.jackson.JsonMethods - -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.deploy.master.RecoveryState -import org.apache.spark.internal.{config, Logging} -import org.apache.spark.util.{ThreadUtils, Utils} - -/** - * This suite tests the fault tolerance of the Spark standalone scheduler, mainly the Master. - * In order to mimic a real distributed cluster more closely, Docker is used. - * Execute using - * ./bin/spark-class org.apache.spark.deploy.FaultToleranceTest - * - * Make sure that the environment includes the following properties in SPARK_DAEMON_JAVA_OPTS: - * - spark.deploy.recoveryMode=ZOOKEEPER - * - spark.deploy.zookeeper.url=172.17.42.1:2181 - * Note that 172.17.42.1 is the default docker ip for the host and 2181 is the default ZK port. - * - * In case of failure, make sure to kill off prior docker containers before restarting: - * docker kill $(docker ps -q) - * - * Unfortunately, due to the Docker dependency this suite cannot be run automatically without a - * working installation of Docker. In addition to having Docker, the following are assumed: - * - Docker can run without sudo (see http://docs.docker.io/en/latest/use/basics/) - * - The docker images tagged spark-test-master and spark-test-worker are built from the - * docker/ directory. Run 'docker/spark-test/build' to generate these. - */ -private object FaultToleranceTest extends App with Logging { - - private val conf = new SparkConf() - private val zkDir = conf.get(config.Deploy.ZOOKEEPER_DIRECTORY).getOrElse("/spark") - - private val masters = ListBuffer[TestMasterInfo]() - private val workers = ListBuffer[TestWorkerInfo]() - private var sc: SparkContext = _ - - private val zk = SparkCuratorUtil.newClient(conf) - - private var numPassed = 0 - private var numFailed = 0 - - private val sparkHome = System.getenv("SPARK_HOME") - assertTrue(sparkHome != null, "Run with a valid SPARK_HOME") - - private val containerSparkHome = "/opt/spark" - private val dockerMountDir = "%s:%s".format(sparkHome, containerSparkHome) - - System.setProperty(config.DRIVER_HOST_ADDRESS.key, "172.17.42.1") // default docker host ip - - private def afterEach(): Unit = { - if (sc != null) { - sc.stop() - sc = null - } - terminateCluster() - - // Clear ZK directories in between tests (for speed purposes) - SparkCuratorUtil.deleteRecursive(zk, zkDir + "/spark_leader") - SparkCuratorUtil.deleteRecursive(zk, zkDir + "/master_status") - } - - test("sanity-basic") { - addMasters(1) - addWorkers(1) - createClient() - assertValidClusterState() - } - - test("sanity-many-masters") { - addMasters(3) - addWorkers(3) - createClient() - assertValidClusterState() - } - - test("single-master-halt") { - addMasters(3) - addWorkers(2) - createClient() - assertValidClusterState() - - killLeader() - delay(30.seconds) - assertValidClusterState() - createClient() - assertValidClusterState() - } - - test("single-master-restart") { - addMasters(1) - addWorkers(2) - createClient() - assertValidClusterState() - - killLeader() - addMasters(1) - delay(30.seconds) - assertValidClusterState() - - killLeader() - addMasters(1) - delay(30.seconds) - assertValidClusterState() - } - - test("cluster-failure") { - addMasters(2) - addWorkers(2) - createClient() - assertValidClusterState() - - terminateCluster() - addMasters(2) - addWorkers(2) - assertValidClusterState() - } - - test("all-but-standby-failure") { - addMasters(2) - addWorkers(2) - createClient() - assertValidClusterState() - - killLeader() - workers.foreach(_.kill()) - workers.clear() - delay(30.seconds) - addWorkers(2) - assertValidClusterState() - } - - test("rolling-outage") { - addMasters(1) - delay() - addMasters(1) - delay() - addMasters(1) - addWorkers(2) - createClient() - assertValidClusterState() - assertTrue(getLeader == masters.head) - - (1 to 3).foreach { _ => - killLeader() - delay(30.seconds) - assertValidClusterState() - assertTrue(getLeader == masters.head) - addMasters(1) - } - } - - private def test(name: String)(fn: => Unit): Unit = { - try { - fn - numPassed += 1 - logInfo("==============================================") - logInfo("Passed: " + name) - logInfo("==============================================") - } catch { - case e: Exception => - numFailed += 1 - logInfo("!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!") - logError("FAILED: " + name, e) - logInfo("!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!") - sys.exit(1) - } - afterEach() - } - - private def addMasters(num: Int): Unit = { - logInfo(s">>>>> ADD MASTERS $num <<<<<") - (1 to num).foreach { _ => masters += SparkDocker.startMaster(dockerMountDir) } - } - - private def addWorkers(num: Int): Unit = { - logInfo(s">>>>> ADD WORKERS $num <<<<<") - val masterUrls = getMasterUrls(masters.toSeq) - (1 to num).foreach { _ => workers += SparkDocker.startWorker(dockerMountDir, masterUrls) } - } - - /** Creates a SparkContext, which constructs a Client to interact with our cluster. */ - private def createClient() = { - logInfo(">>>>> CREATE CLIENT <<<<<") - if (sc != null) { sc.stop() } - // Counter-hack: Because of a hack in SparkEnv#create() that changes this - // property, we need to reset it. - System.setProperty(config.DRIVER_PORT.key, "0") - sc = new SparkContext(getMasterUrls(masters.toSeq), "fault-tolerance", containerSparkHome) - } - - private def getMasterUrls(masters: Seq[TestMasterInfo]): String = { - "spark://" + masters.map(master => master.ip + ":7077").mkString(",") - } - - private def getLeader: TestMasterInfo = { - val leaders = masters.filter(_.state == RecoveryState.ALIVE) - assertTrue(leaders.size == 1) - leaders(0) - } - - private def killLeader(): Unit = { - logInfo(">>>>> KILL LEADER <<<<<") - masters.foreach(_.readState()) - val leader = getLeader - masters -= leader - leader.kill() - } - - private def delay(secs: Duration = 5.seconds) = Thread.sleep(secs.toMillis) - - private def terminateCluster(): Unit = { - logInfo(">>>>> TERMINATE CLUSTER <<<<<") - masters.foreach(_.kill()) - workers.foreach(_.kill()) - masters.clear() - workers.clear() - } - - /** This includes Client retry logic, so it may take a while if the cluster is recovering. */ - private def assertUsable() = { - val f = Future { - try { - val res = sc.parallelize(0 until 10).collect() - assertTrue(res.toList == (0 until 10).toList) - true - } catch { - case e: Exception => - logError("assertUsable() had exception", e) - e.printStackTrace() - false - } - } - - // Avoid waiting indefinitely (e.g., we could register but get no executors). - assertTrue(ThreadUtils.awaitResult(f, 2.minutes)) - } - - /** - * Asserts that the cluster is usable and that the expected masters and workers - * are all alive in a proper configuration (e.g., only one leader). - */ - private def assertValidClusterState() = { - logInfo(">>>>> ASSERT VALID CLUSTER STATE <<<<<") - assertUsable() - var numAlive = 0 - var numStandby = 0 - var numLiveApps = 0 - var liveWorkerIPs: Seq[String] = List() - - def stateValid(): Boolean = { - workers.map(_.ip).forall(liveWorkerIPs.contains) && - numAlive == 1 && numStandby == masters.size - 1 && numLiveApps >= 1 - } - - val f = Future { - try { - while (!stateValid()) { - Thread.sleep(1000) - - numAlive = 0 - numStandby = 0 - numLiveApps = 0 - - masters.foreach(_.readState()) - - for (master <- masters) { - master.state match { - case RecoveryState.ALIVE => - numAlive += 1 - liveWorkerIPs = master.liveWorkerIPs - case RecoveryState.STANDBY => - numStandby += 1 - case _ => // ignore - } - - numLiveApps += master.numLiveApps - } - } - true - } catch { - case e: Exception => - logError("assertValidClusterState() had exception", e) - false - } - } - - try { - assertTrue(ThreadUtils.awaitResult(f, 2.minutes)) - } catch { - case e: TimeoutException => - logError("Master states: " + masters.map(_.state)) - logError("Num apps: " + numLiveApps) - logError("IPs expected: " + workers.map(_.ip) + " / found: " + liveWorkerIPs) - throw new RuntimeException("Failed to get into acceptable cluster state after 2 min.", e) - } - } - - private def assertTrue(bool: Boolean, message: String = ""): Unit = { - if (!bool) { - throw new IllegalStateException("Assertion failed: " + message) - } - } - - logInfo("Ran %s tests, %s passed and %s failed".format(numPassed + numFailed, numPassed, - numFailed)) -} - -private class TestMasterInfo(val ip: String, val dockerId: DockerId, val logFile: File) - extends Logging { - - implicit val formats: Formats = org.json4s.DefaultFormats - var state: RecoveryState.Value = _ - var liveWorkerIPs: List[String] = _ - var numLiveApps = 0 - - logDebug("Created master: " + this) - - def readState(): Unit = { - try { - val masterStream = new InputStreamReader( - new URL("http://%s:8080/json".format(ip)).openStream, StandardCharsets.UTF_8) - val json = JsonMethods.parse(masterStream) - - val workers = json \ "workers" - val liveWorkers = workers.children.filter(w => (w \ "state").extract[String] == "ALIVE") - // Extract the worker IP from "webuiaddress" (rather than "host") because the host name - // on containers is a weird hash instead of the actual IP address. - liveWorkerIPs = liveWorkers.map { - w => (w \ "webuiaddress").extract[String].stripPrefix("http://").stripSuffix(":8081") - } - - numLiveApps = (json \ "activeapps").children.size - - val status = json \\ "status" - val stateString = status.extract[String] - state = RecoveryState.values.filter(state => state.toString == stateString).head - } catch { - case e: Exception => - // ignore, no state update - logWarning("Exception", e) - } - } - - def kill(): Unit = { Docker.kill(dockerId) } - - override def toString: String = - "[ip=%s, id=%s, logFile=%s, state=%s]". - format(ip, dockerId.id, logFile.getAbsolutePath, state) -} - -private class TestWorkerInfo(val ip: String, val dockerId: DockerId, val logFile: File) - extends Logging { - - implicit val formats: Formats = org.json4s.DefaultFormats - - logDebug("Created worker: " + this) - - def kill(): Unit = { Docker.kill(dockerId) } - - override def toString: String = - "[ip=%s, id=%s, logFile=%s]".format(ip, dockerId, logFile.getAbsolutePath) -} - -private object SparkDocker { - def startMaster(mountDir: String): TestMasterInfo = { - val cmd = Docker.makeRunCmd("spark-test-master", mountDir = mountDir) - val (ip, id, outFile) = startNode(cmd) - new TestMasterInfo(ip, id, outFile) - } - - def startWorker(mountDir: String, masters: String): TestWorkerInfo = { - val cmd = Docker.makeRunCmd("spark-test-worker", args = masters, mountDir = mountDir) - val (ip, id, outFile) = startNode(cmd) - new TestWorkerInfo(ip, id, outFile) - } - - private def startNode(dockerCmd: ProcessBuilder) : (String, DockerId, File) = { - val ipPromise = Promise[String]() - val outFile = File.createTempFile("fault-tolerance-test", "", Utils.createTempDir()) - val outStream: FileWriter = new FileWriter(outFile) - def findIpAndLog(line: String): Unit = { - if (line.startsWith("CONTAINER_IP=")) { - val ip = line.split("=")(1) - ipPromise.success(ip) - } - - outStream.write(line + "\n") - outStream.flush() - } - - dockerCmd.run(ProcessLogger(findIpAndLog _)) - val ip = ThreadUtils.awaitResult(ipPromise.future, 30.seconds) - val dockerId = Docker.getLastProcessId - (ip, dockerId, outFile) - } -} - -private class DockerId(val id: String) { - override def toString: String = id -} - -private object Docker extends Logging { - def makeRunCmd(imageTag: String, args: String = "", mountDir: String = ""): ProcessBuilder = { - val mountCmd = if (mountDir != "") { " -v " + mountDir } else "" - - val cmd = "docker run -privileged %s %s %s".format(mountCmd, imageTag, args) - logDebug("Run command: " + cmd) - cmd - } - - def kill(dockerId: DockerId) : Unit = { - "docker kill %s".format(dockerId.id).! - } - - def getLastProcessId: DockerId = { - var id: String = null - "docker ps -l -q".!(ProcessLogger(line => id = line)) - new DockerId(id) - } -} 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 be11c23f306e..bd07a0ade523 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, MDC} +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.internal.LogKeys.WORKER_URL import org.apache.spark.rpc._ @@ -64,7 +64,7 @@ private[spark] class WorkerWatcher( } override def receive: PartialFunction[Any, Unit] = { - case e => logWarning(s"Received unexpected message: $e") + case e => logWarning(log"Received unexpected message: ${MDC(LogKeys.ERROR, e)}") } override def onConnected(remoteAddress: RpcAddress): Unit = { 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 d0a202cb7951..022a9c2ba9a8 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -436,8 +436,8 @@ private[spark] class IndexShuffleBlockResolver( if (checksumTmp.exists()) { try { if (!checksumTmp.delete()) { - logError(s"Failed to delete temporary checksum file " + - s"at ${checksumTmp.getAbsolutePath}") + logError(log"Failed to delete temporary checksum file at " + + log"${MDC(LogKeys.FILE_ABSOLUTE_PATH, checksumTmp.getAbsolutePath)}") } } catch { case e: Exception => 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 2afab01ec7b0..20b3d9444884 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 +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.sql.execution.streaming.AbstractFileContextBasedCheckpointFileManager import org.apache.spark.sql.execution.streaming.CheckpointFileManager.CancellableFSDataOutputStream @@ -36,7 +36,7 @@ class AbortableStreamBasedCheckpointFileManager(path: Path, hadoopConf: Configur s" an fs (path: $path) with abortable stream support") } - logInfo(s"Writing atomically to $path based on abortable stream") + logInfo(log"Writing atomically to ${MDC(LogKeys.PATH, path)} based on abortable stream") class AbortableStreamBasedFSDataOutputStream( fsDataOutputStream: FSDataOutputStream, 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 b523bd750836..8b9d7e802555 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 @@ -503,8 +503,8 @@ class LogisticRegression @Since("1.2.0") ( tol, fitIntercept, maxBlockSizeInMB) if (dataset.storageLevel != StorageLevel.NONE) { - instr.logWarning(s"Input instances will be standardized, blockified to blocks, and " + - s"then cached during training. Be careful of double caching!") + instr.logWarning(log"Input instances will be standardized, blockified to blocks, and " + + log"then cached during training. Be careful of double caching!") } val instances = dataset.select( @@ -569,8 +569,8 @@ class LogisticRegression @Since("1.2.0") ( val isConstantLabel = histogram.count(_ != 0.0) == 1 if ($(fitIntercept) && isConstantLabel && !usingBoundConstrainedOptimization) { - instr.logWarning(s"All labels are the same value and fitIntercept=true, so the " + - s"coefficients will be zeros. Training is not needed.") + instr.logWarning(log"All labels are the same value and fitIntercept=true, so the " + + log"coefficients will be zeros. Training is not needed.") val constantLabelIndex = Vectors.dense(histogram).argmax val coefMatrix = new SparseMatrix(numCoefficientSets, numFeatures, new Array[Int](numCoefficientSets + 1), Array.emptyIntArray, Array.emptyDoubleArray, @@ -584,8 +584,8 @@ class LogisticRegression @Since("1.2.0") ( } if (!$(fitIntercept) && isConstantLabel) { - instr.logWarning(s"All labels belong to a single class and fitIntercept=false. It's a " + - s"dangerous ground, so the algorithm may not converge.") + instr.logWarning(log"All labels belong to a single class and fitIntercept=false. It's a " + + log"dangerous ground, so the algorithm may not converge.") } val featuresMean = summarizer.mean.toArray 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 4b5f9be3193f..3bec02d12b1c 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 @@ -528,8 +528,9 @@ private[spark] class ApplicationMaster( } catch { case e: SparkException if e.getCause().isInstanceOf[TimeoutException] => logError( - s"SparkContext did not initialize after waiting for $totalWaitTime ms. " + - "Please check earlier log output for errors. Failing the application.") + log"""SparkContext did not initialize after waiting for + |${MDC(LogKeys.WAIT_TIME, totalWaitTime)} ms. + | Please check earlier log output for errors. Failing the application.""".stripMargin) finish(FinalApplicationStatus.FAILED, ApplicationMaster.EXIT_SC_NOT_INITED, "Timed out waiting for SparkContext.") @@ -690,7 +691,7 @@ private[spark] class ApplicationMaster( } } catch { case ioe: IOException => - logError("Failed to cleanup staging dir " + stagingDirPath, ioe) + logError(log"Failed to cleanup staging dir ${MDC(LogKeys.PATH, stagingDirPath)}", ioe) } } @@ -736,7 +737,8 @@ private[spark] class ApplicationMaster( override def run(): Unit = { try { if (!Modifier.isStatic(mainMethod.getModifiers)) { - logError(s"Could not find static main method in object ${args.userClass}") + logError(log"Could not find static main method in object " + + log"${MDC(LogKeys.CLASS_NAME, args.userClass)}") finish(FinalApplicationStatus.FAILED, ApplicationMaster.EXIT_EXCEPTION_USER_CLASS) } else { mainMethod.invoke(null, userArgs.toArray) @@ -866,7 +868,8 @@ private[spark] class ApplicationMaster( finish(FinalApplicationStatus.FAILED, exitCode) } } else { - logError(s"Application Master lost connection with driver! Shutting down. $remoteAddress") + logError(log"Application Master lost connection with driver! Shutting down. " + + log"${MDC(LogKeys.REMOTE_ADDRESS, remoteAddress)}") finish(FinalApplicationStatus.FAILED, ApplicationMaster.EXIT_DISCONNECTED) } } 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 0567d8efb85e..c5673fdcd2be 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 @@ -620,8 +620,9 @@ private[spark] class Client( // If we passed in a keytab, make sure we copy the keytab to the staging directory on // HDFS, and setup the relevant environment vars, so the AM can login again. amKeytabFileName.foreach { kt => - logInfo("To enable the AM to login from keytab, credentials are being copied over to the AM" + - " via the YARN Secure Distributed Cache.") + logInfo(log"To enable the AM to login from keytab, " + + log"credentials are being copied over to the AM " + + log"via the YARN Secure Distributed Cache.") val (_, localizedPath) = distribute(keytab, destName = Some(kt), appMasterOnly = true) @@ -1385,7 +1386,7 @@ private[spark] class Client( val YarnAppReport(appState, finalState, diags) = monitorApplication() if (appState == YarnApplicationState.FAILED || finalState == FinalApplicationStatus.FAILED) { diags.foreach { err => - logError(s"Application diagnostics message: $err") + logError(log"Application diagnostics message: ${MDC(LogKeys.MESSAGE, err)}") } throw new SparkException(s"Application $appId finished with failed status") } 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 26be1ff89314..119657409899 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 @@ -120,7 +120,7 @@ private[spark] class YarnClientSchedulerBackend( logError(log"YARN application has exited unexpectedly with state " + log"${MDC(APP_STATE, state)}! Check the YARN application logs for more details.") diags.foreach { err => - logError(s"Diagnostics message: $err") + logError(log"Diagnostics message: ${MDC(LogKeys.ERROR, err)}") } allowInterrupt = false sc.stop() diff --git a/scalastyle-config.xml b/scalastyle-config.xml index cd5a576c086f..30192b8e3925 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -155,6 +155,14 @@ This file is divided into 3 sections: + + log(?:Info|Warning|Error)\((?:".*"\.format\(.*\)|s".*(?:\$|\+\s*[^\s"]).*"\)) + + Logging message should use log"..." instead of s"..." and variables should be wrapped in `MDC`s. + Refer to Structured Logging Framework guidelines in the file `internal/Logging.scala`. + + + spark(.sqlContext)?.sparkContext.hadoopConfiguration DEFAULT_JVM_HUGE_METHOD_LIMIT) { - logInfo("Generated method too long to be JIT compiled: " + - log"${MDC(CLASS_NAME, cf.getThisClassName)}.${MDC(METHOD_NAME, method.getName)} " + - log"is ${MDC(BYTECODE_SIZE, byteCodeSize)} bytes") + logInfo(log"Generated method too long to be JIT compiled: " + + log"${MDC(LogKeys.CLASS_NAME, cf.getThisClassName)}." + + log"${MDC(LogKeys.METHOD_NAME, method.getName)} is " + + log"${MDC(LogKeys.BYTECODE_SIZE, byteCodeSize)} bytes") } byteCodeSize 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 9370b3d8d1d7..dbb8deb24400 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 @@ -61,8 +61,10 @@ object ShufflePartitionsUtil extends Logging { val targetSize = maxTargetSize.min(advisoryTargetSize).max(minPartitionSize) val shuffleIds = mapOutputStatistics.flatMap(_.map(_.shuffleId)).mkString(", ") - logInfo(s"For shuffle($shuffleIds), advisory target size: $advisoryTargetSize, " + - s"actual target size $targetSize, minimum partition size: $minPartitionSize") + logInfo(log"For shuffle(${MDC(LogKeys.SHUFFLE_ID, shuffleIds)}, advisory target size: " + + log"${MDC(LogKeys.TARGET_SIZE, advisoryTargetSize)}, actual target size " + + log"${MDC(LogKeys.TARGET_SIZE, targetSize)}, minimum partition size: " + + log"${MDC(LogKeys.PARTITION_SIZE, minPartitionSize)}") // If `inputPartitionSpecs` are all empty, it means skew join optimization is not applied. if (inputPartitionSpecs.forall(_.isEmpty)) { 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 75a38b8ea622..7d80cc272810 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 +import org.apache.spark.internal.{Logging, LogKeys, MDC} 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} @@ -52,7 +52,8 @@ class PythonStreamingPartitionReaderFactory( val block = SparkEnv.get.blockManager.get[InternalRow](part.blockId.get) .map(_.data.asInstanceOf[Iterator[InternalRow]]) if (block.isEmpty) { - logWarning(s"Prefetched block ${part.blockId} for Python data source not found.") + logWarning(log"Prefetched block ${MDC(LogKeys.BLOCK_ID, part.blockId)} " + + log"for Python data source not found.") } block } else None diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala index a512b34db345..9d76cb063518 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala @@ -214,7 +214,8 @@ class PythonStreamingSourceRunner( * Stop the python worker process and invoke stop() on stream reader. */ def stop(): Unit = { - logInfo(s"Stopping streaming runner for module: $workerModule.") + logInfo(log"Stopping streaming runner for module: " + + log"${MDC(LogKeys.MODULE_NAME, workerModule)}.") try { pythonWorkerFactory.foreach { factory => pythonWorker.foreach { worker => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala index 3842ed574355..c440ec451b72 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala @@ -352,8 +352,9 @@ abstract class ProgressContext( metrics = sourceMetrics ) } - logInfo(s"Extracting source progress metrics for source=${source.toString} took " + - s"duration_ms=$duration") + logInfo(log"Extracting source progress metrics for source=" + + log"${MDC(LogKeys.SOURCE, source.toString)} " + + log"took duration_ms=${MDC(LogKeys.DURATION, duration)}") result } } @@ -368,8 +369,8 @@ abstract class ProgressContext( SinkProgress(sink.toString, sinkOutput, sinkMetrics) } - logInfo(s"Extracting sink progress metrics for sink=${sink.toString} took " + - s"duration_ms=$duration") + logInfo(log"Extracting sink progress metrics for sink=${MDC(LogKeys.SINK, sink.toString)} " + + log"took duration_ms=${MDC(LogKeys.DURATION, duration)}") result } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 420deda3e017..77c08ce9bf6f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -32,7 +32,7 @@ import com.google.common.util.concurrent.UncheckedExecutionException import org.apache.hadoop.fs.Path import org.apache.spark.{JobArtifactSet, SparkContext, SparkException, SparkThrowable} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.internal.LogKeys.{CHECKPOINT_PATH, CHECKPOINT_ROOT, PATH, PRETTY_ID_STRING, SPARK_DATA_STREAM} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan @@ -322,7 +322,8 @@ abstract class StreamExecution( if (state.compareAndSet(INITIALIZING, ACTIVE)) { // Log logical plan at the start of the query to help debug issues related to // plan changes. - logInfo(s"Finish initializing with logical plan:\n$logicalPlan") + logInfo(log"Finish initializing with logical plan:\n" + + log"${MDC(LogKeys.LOGICAL_PLAN_LEAVES, logicalPlan)}") // Unblock `awaitInitialization` initializationLatch.countDown() @@ -372,7 +373,8 @@ abstract class StreamExecution( case _ => None } - logError(s"Query $prettyIdString terminated with error", e) + logError(log"Query ${MDC(LogKeys.PRETTY_ID_STRING, prettyIdString)} " + + log"terminated with error", e) getLatestExecutionContext().updateStatusMessage(s"Terminated with exception: $message") // Rethrow the fatal errors to allow the user using `Thread.UncaughtExceptionHandler` to // handle them 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 fa49da5feeed..a56606f8c1cb 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 @@ -445,7 +445,8 @@ class ContinuousExecution( */ def stopInNewThread(error: Throwable): Unit = { if (failure.compareAndSet(null, error)) { - logError(s"Query $prettyIdString received exception $error") + logError(log"Query ${MDC(LogKeys.PRETTY_ID_STRING, prettyIdString)} received exception " + + log"${MDC(LogKeys.ERROR, error)}") stopInNewThread() } } 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 5d54b5754915..d5daa9a875f8 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,8 +18,8 @@ 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.MDC import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.connector.write.DataWriter @@ -89,9 +89,11 @@ class ContinuousWriteRDD(var prev: RDD[InternalRow], writerFactory: StreamingDat })(catchBlock = { // If there is an error, abort this writer. We enter this callback in the middle of // rethrowing an exception, so compute() will stop executing at this point. - logError(s"Writer for partition ${context.partitionId()} is aborting.") + logError(log"Writer for partition ${MDC(LogKeys.PARTITION_ID, context.partitionId())} " + + log"is aborting.") if (dataWriter != null) dataWriter.abort() - logError(s"Writer for partition ${context.partitionId()} aborted.") + logError(log"Writer for partition ${MDC(LogKeys.PARTITION_ID, context.partitionId())} " + + log"aborted.") }, finallyBlock = { if (dataWriter != null) dataWriter.close() }) 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 23f867d3e6c0..20df67b25bfe 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 @@ -25,7 +25,7 @@ import com.google.common.io.ByteStreams import org.apache.commons.io.IOUtils import org.apache.hadoop.fs.{FSError, Path} -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.internal.LogKeys._ import org.apache.spark.io.CompressionCodec import org.apache.spark.sql.errors.QueryExecutionErrors @@ -176,7 +176,8 @@ class StateStoreChangelogWriterV1( } catch { case e: Throwable => abort() - logError(s"Fail to commit changelog file $file because of exception $e") + logError(log"Fail to commit changelog file ${MDC(LogKeys.FILE_NAME, file)} " + + log"because of exception ${MDC(LogKeys.EXCEPTION, e)}") throw e } finally { backingFileStream = null @@ -255,7 +256,8 @@ class StateStoreChangelogWriterV2( } catch { case e: Throwable => abort() - logError(s"Fail to commit changelog file $file because of exception $e") + logError(log"Fail to commit changelog file ${MDC(LogKeys.FILE_NAME, file)} " + + log"because of exception ${MDC(LogKeys.EXCEPTION, e)}") throw e } finally { backingFileStream = null 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 b86e996a3408..51a5e88aa633 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, MDC} +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.{DataFrame, Row, SQLContext} import org.apache.spark.sql.catalyst.util.CharVarcharUtils @@ -83,7 +83,7 @@ private[hive] class SparkExecuteStatementOperation( val sparkType = new StructType().add("Result", "string") SparkExecuteStatementOperation.toTTableSchema(sparkType) } else { - logInfo(s"Result Schema: ${result.schema.sql}") + logInfo(log"Result Schema: ${MDC(LogKeys.SCHEMA, result.schema.sql)}") SparkExecuteStatementOperation.toTTableSchema(result.schema) } } @@ -126,8 +126,8 @@ private[hive] class SparkExecuteStatementOperation( override def runInternal(): Unit = { setState(OperationState.PENDING) logInfo( - log"Submitting query '${MDC(REDACTED_STATEMENT, redactedStatement)}' with " + - log"${MDC(STATEMENT_ID, statementId)}") + log"Submitting query '${MDC(LogKeys.REDACTED_STATEMENT, redactedStatement)}' with " + + log"${MDC(LogKeys.STATEMENT_ID, statementId)}") HiveThriftServer2.eventManager.onStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, @@ -215,8 +215,8 @@ private[hive] class SparkExecuteStatementOperation( synchronized { if (getStatus.getState.isTerminal) { logInfo( - log"Query with ${MDC(STATEMENT_ID, statementId)} in terminal state " + - log"before it started running") + log"Query with ${MDC(LogKeys.STATEMENT_ID, statementId)} in terminal state " + + log"before it started running") return } else { logInfo(log"Running query with ${MDC(STATEMENT_ID, statementId)}") @@ -289,8 +289,8 @@ private[hive] class SparkExecuteStatementOperation( synchronized { if (!getStatus.getState.isTerminal) { logInfo( - log"Query with ${MDC(STATEMENT_ID, statementId)} timed out after " + - log"${MDC(TIMEOUT, timeout)} seconds") + log"Query with ${MDC(LogKeys.STATEMENT_ID, statementId)} timed out " + + log"after ${MDC(LogKeys.TIMEOUT, timeout)} seconds") setState(OperationState.TIMEDOUT) cleanup() HiveThriftServer2.eventManager.onStatementTimeout(statementId) 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 53a94a128c0e..9cf31d99ccfa 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 +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.sql.SQLContext /** @@ -53,7 +53,7 @@ private[hive] class SparkGetFunctionsOperation( // Do not change cmdStr. It's used for Hive auditing and authorization. val cmdStr = s"catalog : $catalogName, schemaPattern : $schemaName" val logMsg = s"Listing functions '$cmdStr, functionName : $functionName'" - logInfo(s"$logMsg with $statementId") + logInfo(log"${MDC(LogKeys.MESSAGE, logMsg)} with ${MDC(LogKeys.STATEMENT_ID, statementId)}") setState(OperationState.RUNNING) // Always use the latest class loader provided by executionHive's state. val executionHiveClassLoader = sqlContext.sharedState.jarClassLoader diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala index 46537f75f1a1..9421c8ae496d 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala @@ -34,7 +34,7 @@ import org.apache.hive.service.auth.HiveAuthFactory import org.apache.hive.service.cli._ import org.apache.hive.service.server.HiveServer2 -import org.apache.spark.internal.SparkLogger +import org.apache.spark.internal.{Logging, LogKeys, MDC, SparkLogger} import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.util.SQLKeywordUtils import org.apache.spark.sql.errors.QueryExecutionErrors @@ -127,7 +127,7 @@ private[thriftserver] trait ReflectedCompositeService { this: AbstractService => invoke(classOf[AbstractService], this, "ensureCurrentState", classOf[STATE] -> STATE.NOTINITED) setAncestorField(this, 3, "hiveConf", hiveConf) invoke(classOf[AbstractService], this, "changeState", classOf[STATE] -> STATE.INITED) - logInfo(s"Service: $getName is inited.") + logInfo(log"Service: ${MDC(LogKeys.SERVICE_NAME, getName)} is inited.") } def startCompositeService(): Unit = { @@ -144,10 +144,10 @@ private[thriftserver] trait ReflectedCompositeService { this: AbstractService => setAncestorField(this, 3, "startTime", startTime) invoke(classOf[AbstractService], this, "ensureCurrentState", classOf[STATE] -> STATE.INITED) invoke(classOf[AbstractService], this, "changeState", classOf[STATE] -> STATE.STARTED) - logInfo(s"Service: $getName is started.") + logInfo(log"Service: ${MDC(LogKeys.SERVICE_NAME, getName)} is started.") } catch { case NonFatal(e) => - logError(s"Error starting services $getName", e) + logError(log"Error starting services ${MDC(LogKeys.SERVICE_NAME, getName)}", e) invoke(classOf[CompositeService], this, "stop", classOf[Int] -> Integer.valueOf(serviceStartCount)) throw HiveThriftServerErrors.failedToStartServiceError(getName, e) 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 6cbc74a75a06..f09f9caf129b 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, MDC} +import org.apache.spark.internal.{Logging, LogKeys, MDC} 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 @@ -102,7 +102,7 @@ class Checkpoint(ssc: StreamingContext, val checkpointTime: Time) assert(framework != null, "Checkpoint.framework is null") assert(graph != null, "Checkpoint.graph is null") assert(checkpointTime != null, "Checkpoint.checkpointTime is null") - logInfo(s"Checkpoint for time $checkpointTime validated") + logInfo(log"Checkpoint for time ${MDC(LogKeys.CHECKPOINT_TIME, checkpointTime)} validated") } } @@ -242,7 +242,8 @@ class CheckpointWriter( while (attempts < MAX_ATTEMPTS && !stopped) { attempts += 1 try { - logInfo(s"Saving checkpoint for time $checkpointTime to file '$checkpointFile'") + logInfo(log"Saving checkpoint for time ${MDC(LogKeys.CHECKPOINT_TIME, checkpointTime)} " + + log"to file '${MDC(LogKeys.CHECKPOINT_FILE, checkpointFile)}'") if (fs == null) { fs = new Path(checkpointDir).getFileSystem(hadoopConf) } @@ -275,15 +276,19 @@ class CheckpointWriter( val allCheckpointFiles = Checkpoint.getCheckpointFiles(checkpointDir, Some(fs)) if (allCheckpointFiles.size > 10) { allCheckpointFiles.take(allCheckpointFiles.size - 10).foreach { file => - logInfo(s"Deleting $file") + logInfo(log"Deleting ${MDC(LogKeys.FILE_NAME, file)}") fs.delete(file, true) } } // All done, print success - logInfo(s"Checkpoint for time $checkpointTime saved to file '$checkpointFile'" + - s", took ${bytes.length} bytes and " + - s"${TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNs)} ms") + logInfo( + log"Checkpoint for time ${MDC(LogKeys.CHECKPOINT_TIME, checkpointTime)} " + + log"saved to file " + + log"'${MDC(LogKeys.CHECKPOINT_FILE, checkpointFile)}', took " + + log"${MDC(LogKeys.BYTE_SIZE, bytes.length)} bytes and " + + log"${MDC(LogKeys.TIME, TimeUnit.NANOSECONDS.toMillis(System.nanoTime() + - startTimeNs))} ms") jobGenerator.onCheckpointCompletion(checkpointTime, clearCheckpointDataLater) return } catch { @@ -304,7 +309,8 @@ class CheckpointWriter( val bytes = Checkpoint.serialize(checkpoint, conf) executor.execute(new CheckpointWriteHandler( checkpoint.checkpointTime, bytes, clearCheckpointDataLater)) - logInfo(s"Submitted checkpoint of time ${checkpoint.checkpointTime} to writer queue") + logInfo(log"Submitted checkpoint of time ${MDC(LogKeys.CHECKPOINT_TIME, + checkpoint.checkpointTime)} to writer queue") } catch { case rej: RejectedExecutionException => logError("Could not submit checkpoint task to the thread pool executor", rej) @@ -316,8 +322,10 @@ class CheckpointWriter( val startTimeNs = System.nanoTime() ThreadUtils.shutdown(executor, FiniteDuration(10, TimeUnit.SECONDS)) - logInfo(s"CheckpointWriter executor terminated? ${executor.isTerminated}," + - s" waited for ${TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNs)} ms.") + logInfo(log"CheckpointWriter executor terminated? " + + log"${MDC(LogKeys.EXECUTOR_STATE, executor.isTerminated)}, waited for " + + log"${MDC(LogKeys.DURATION, TimeUnit.NANOSECONDS.toMillis( + System.nanoTime() - startTimeNs))} ms.") stopped = true } } @@ -357,15 +365,17 @@ object CheckpointReader extends Logging { } // Try to read the checkpoint files in the order - logInfo(s"Checkpoint files found: ${checkpointFiles.mkString(",")}") + logInfo(log"Checkpoint files found: " + + log"${MDC(LogKeys.CHECKPOINT_FILE, checkpointFiles.mkString(","))}") var readError: Exception = null checkpointFiles.foreach { file => - logInfo(s"Attempting to load checkpoint from file $file") + logInfo(log"Attempting to load checkpoint from file ${MDC(LogKeys.FILE_NAME, file)}") try { val fis = fs.open(file) val cp = Checkpoint.deserialize(fis, conf) - logInfo(s"Checkpoint successfully loaded from file $file") - logInfo(s"Checkpoint was generated at time ${cp.checkpointTime}") + logInfo(log"Checkpoint successfully loaded from file ${MDC(LogKeys.FILE_NAME, file)}") + logInfo(log"Checkpoint was generated at time " + + log"${MDC(LogKeys.CHECKPOINT_TIME, cp.checkpointTime)}") return Some(cp) } catch { case e: Exception => 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 30bd30329283..3898d00c7b06 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -725,7 +725,8 @@ class StreamingContext private[streaming] ( private def stopOnShutdown(): Unit = { val stopGracefully = conf.get(STOP_GRACEFULLY_ON_SHUTDOWN) - logInfo(s"Invoking stop(stopGracefully=$stopGracefully) from shutdown hook") + logInfo(log"Invoking stop(stopGracefully=" + + log"${MDC(LogKeys.STOP_SITE_SHORT_FORM, stopGracefully)}) from shutdown hook") // Do not stop SparkContext, let its own shutdown hook stop it stop(stopSparkContext = false, stopGracefully = stopGracefully) } 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 38f55f80657b..9b86d8b6df82 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, MDC} +import org.apache.spark.internal.{Logging, LogKeys, MDC} 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} @@ -201,7 +201,8 @@ abstract class DStream[T: ClassTag] ( // Set the checkpoint interval to be slideDuration or 10 seconds, which ever is larger if (mustCheckpoint && checkpointDuration == null) { checkpointDuration = slideDuration * math.ceil(Seconds(10) / slideDuration).toInt - logInfo(s"Checkpoint interval automatically set to $checkpointDuration") + logInfo(log"Checkpoint interval automatically set to " + + log"${MDC(LogKeys.CHECKPOINT_INTERVAL, checkpointDuration)}") } // Set the minimum value of the rememberDuration if not already set @@ -277,11 +278,11 @@ abstract class DStream[T: ClassTag] ( dependencies.foreach(_.validateAtStart()) - logInfo(s"Slide time = $slideDuration") - logInfo(s"Storage level = ${storageLevel.description}") - logInfo(s"Checkpoint interval = $checkpointDuration") - logInfo(s"Remember interval = $rememberDuration") - logInfo(s"Initialized and validated $this") + logInfo(log"Slide time = ${MDC(LogKeys.SLIDE_DURATION, slideDuration)}") + logInfo(log"Storage level = ${MDC(LogKeys.STORAGE_LEVEL, storageLevel.description)}") + logInfo(log"Checkpoint interval = ${MDC(LogKeys.CHECKPOINT_INTERVAL, checkpointDuration)}") + logInfo(log"Remember interval = ${MDC(LogKeys.INTERVAL, rememberDuration)}") + logInfo(log"Initialized and validated ${MDC(LogKeys.DSTREAM, this)}") } private[streaming] def setContext(s: StreamingContext): Unit = { @@ -289,7 +290,7 @@ abstract class DStream[T: ClassTag] ( throw new SparkException(s"Context must not be set again for $this") } ssc = s - logInfo(s"Set context for $this") + logInfo(log"Set context for ${MDC(LogKeys.CONTEXT, this)}") dependencies.foreach(_.setContext(ssc)) } @@ -304,7 +305,9 @@ abstract class DStream[T: ClassTag] ( private[streaming] def remember(duration: Duration): Unit = { if (duration != null && (rememberDuration == null || duration > rememberDuration)) { rememberDuration = duration - logInfo(s"Duration for remembering RDDs set to $rememberDuration for $this") + logInfo(log"Duration for remembering RDDs set to " + + log"${MDC(LogKeys.DURATION, rememberDuration)} for " + + log"${MDC(LogKeys.DSTREAM, this.toString)}") } dependencies.foreach(_.remember(parentRememberDuration)) } @@ -314,8 +317,10 @@ abstract class DStream[T: ClassTag] ( if (!isInitialized) { throw new SparkException (this.toString + " has not been initialized") } else if (time <= zeroTime || ! (time - zeroTime).isMultipleOf(slideDuration)) { - logInfo(s"Time $time is invalid as zeroTime is $zeroTime" + - s" , slideDuration is $slideDuration and difference is ${time - zeroTime}") + logInfo(log"Time ${MDC(LogKeys.TIME, time)} is invalid as zeroTime is " + + log"${MDC(LogKeys.ZERO_TIME, zeroTime)}, slideDuration is " + + log"${MDC(LogKeys.SLIDE_DURATION, slideDuration)} and difference is " + + log"${MDC(LogKeys.TIME, time - zeroTime)}") false } else { logDebug(s"Time $time is valid") @@ -353,7 +358,8 @@ abstract class DStream[T: ClassTag] ( } if (checkpointDuration != null && (time - zeroTime).isMultipleOf(checkpointDuration)) { newRDD.checkpoint() - logInfo(s"Marking RDD ${newRDD.id} for time $time for checkpointing") + logInfo(log"Marking RDD ${MDC(LogKeys.RDD_ID, newRDD.id)} for time " + + log"${MDC(LogKeys.TIME, time)} for checkpointing") } generatedRDDs.put(time, newRDD) } @@ -461,7 +467,8 @@ abstract class DStream[T: ClassTag] ( // Explicitly remove blocks of BlockRDD rdd match { case b: BlockRDD[_] => - logInfo(s"Removing blocks of RDD $b of time $time") + logInfo(log"Removing blocks of RDD ${MDC(LogKeys.RDD_ID, b)} " + + log"of time ${MDC(LogKeys.TIME, time)}") b.removeBlocks() case _ => } @@ -898,8 +905,10 @@ abstract class DStream[T: ClassTag] ( fromTime.floor(slideDuration, zeroTime) } - logInfo(s"Slicing from $fromTime to $toTime" + - s" (aligned to $alignedFromTime and $alignedToTime)") + logInfo(log"Slicing from ${MDC(LogKeys.FROM_TIME, fromTime)} to " + + log"${MDC(LogKeys.TO_TIME, toTime)}" + + log" (aligned to ${MDC(LogKeys.ALIGNED_FROM_TIME, alignedFromTime)} and " + + log"${MDC(LogKeys.ALIGNED_TO_TIME, alignedToTime)})") alignedFromTime.to(alignedToTime, slideDuration).flatMap { time => if (time >= zeroTime) getOrCompute(time) else None 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 883d56c012f6..34b079219c99 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 +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.receiver.Receiver @@ -56,7 +56,7 @@ class SocketReceiver[T: ClassTag]( def onStart(): Unit = { - logInfo(s"Connecting to $host:$port") + logInfo(log"Connecting to ${MDC(LogKeys.HOST, host)}:${MDC(LogKeys.PORT, port)}") try { socket = new Socket(host, port) } catch { @@ -64,7 +64,7 @@ class SocketReceiver[T: ClassTag]( restart(s"Error connecting to $host:$port", e) return } - logInfo(s"Connected to $host:$port") + logInfo(log"Connected to ${MDC(LogKeys.HOST, host)}:${MDC(LogKeys.PORT, port)}") // Start the thread that receives data over a connection new Thread("Socket Receiver") { @@ -79,7 +79,7 @@ class SocketReceiver[T: ClassTag]( if (socket != null) { socket.close() socket = null - logInfo(s"Closed socket to $host:$port") + logInfo(log"Closed socket to ${MDC(LogKeys.HOST, host)}:${MDC(LogKeys.PORT, port)}") } } } 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 79bfd8674b44..7cc08b421f78 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, MDC} +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.internal.LogKeys.{DELAY, ERROR, MESSAGE, STREAM_ID} import org.apache.spark.storage.StreamBlockId import org.apache.spark.util.{ThreadUtils, Utils} @@ -145,10 +145,10 @@ private[streaming] abstract class ReceiverSupervisor( def startReceiver(): Unit = synchronized { try { if (onReceiverStart()) { - logInfo(s"Starting receiver $streamId") + logInfo(log"Starting receiver ${MDC(LogKeys.STREAM_ID, streamId)}") receiverState = Started receiver.onStart() - logInfo(s"Called receiver $streamId onStart") + logInfo(log"Called receiver ${MDC(LogKeys.STREAM_ID, streamId)} onStart") } else { // The driver refused us stop("Registered unsuccessfully because Driver refused to start receiver " + streamId, None) @@ -162,7 +162,8 @@ private[streaming] abstract class ReceiverSupervisor( /** Stop receiver */ def stopReceiver(message: String, error: Option[Throwable]): Unit = synchronized { try { - logInfo("Stopping receiver with message: " + message + ": " + error.getOrElse("")) + logInfo(log"Stopping receiver with message: ${MDC(LogKeys.MESSAGE, message)}: " + + log"${MDC(LogKeys.ERROR, error.getOrElse(""))}") receiverState match { case Initialized => logWarning("Skip stopping receiver because it has not yet stared") 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 47beb4521950..9a7ea7db8c48 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 @@ -85,7 +85,7 @@ private[streaming] class ReceiverSupervisorImpl( logDebug("Received delete old batch signal") cleanupOldBlocks(threshTime) case UpdateRateLimit(eps) => - logInfo(s"Received a new rate limit: $eps.") + logInfo(log"Received a new rate limit: ${MDC(LogKeys.RATE_LIMIT, eps)}.") registeredBlockGenerators.asScala.foreach { bg => bg.updateRate(eps) } @@ -195,10 +195,10 @@ private[streaming] class ReceiverSupervisorImpl( } override protected def onReceiverStop(message: String, error: Option[Throwable]): Unit = { - logInfo("Deregistering receiver " + streamId) + logInfo(log"Deregistering receiver ${MDC(LogKeys.STREAM_ID, streamId)}") val errorString = error.map(Throwables.getStackTraceAsString).getOrElse("") trackerEndpoint.askSync[Boolean](DeregisterReceiver(streamId, message, errorString)) - logInfo("Stopped receiver " + streamId) + logInfo(log"Stopped receiver ${MDC(LogKeys.RECEIVER_ID, streamId)}") } override def createBlockGenerator( 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 5aa2a9df3ba8..00750df0b8d0 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 +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.internal.config.DECOMMISSION_ENABLED import org.apache.spark.internal.config.Streaming._ import org.apache.spark.resource.ResourceProfile @@ -75,8 +75,10 @@ private[streaming] class ExecutorAllocationManager( def start(): Unit = { timer.start() - logInfo(s"ExecutorAllocationManager started with " + - s"ratios = [$scalingUpRatio, $scalingDownRatio] and interval = $scalingIntervalSecs sec") + logInfo(log"ExecutorAllocationManager started with ratios = " + + log"[${MDC(LogKeys.SCALING_UP_RATIO, scalingUpRatio)}, " + + log"${MDC(LogKeys.SCALING_DOWN_RATIO, scalingDownRatio)}] and interval = " + + log"${MDC(LogKeys.SCALING_INTERVAL_SECS, scalingIntervalSecs)} sec") } def stop(): Unit = { @@ -89,11 +91,14 @@ private[streaming] class ExecutorAllocationManager( * batch statistics. */ private def manageAllocation(): Unit = synchronized { - logInfo(s"Managing executor allocation with ratios = [$scalingUpRatio, $scalingDownRatio]") + logInfo(log"Managing executor allocation with ratios = [" + + log"${MDC(LogKeys.SCALING_UP_RATIO, scalingUpRatio)}, " + + log"${MDC(LogKeys.SCALING_DOWN_RATIO, scalingDownRatio)}]") if (batchProcTimeCount > 0) { val averageBatchProcTime = batchProcTimeSum / batchProcTimeCount val ratio = averageBatchProcTime.toDouble / batchDurationMs - logInfo(s"Average: $averageBatchProcTime, ratio = $ratio" ) + logInfo(log"Average: ${MDC(LogKeys.AVG_BATCH_PROC_TIME, averageBatchProcTime)}, " + + log"ratio = ${MDC(LogKeys.RATIO, ratio)}") if (ratio >= scalingUpRatio) { logDebug("Requesting executors") val numNewExecutors = math.max(math.round(ratio).toInt, 1) @@ -119,7 +124,8 @@ private[streaming] class ExecutorAllocationManager( Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> targetTotalExecutors), Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> 0), Map.empty) - logInfo(s"Requested total $targetTotalExecutors executors") + logInfo(log"Requested total ${MDC(LogKeys.TARGET_NUM_EXECUTOR, + targetTotalExecutors)} executors") } /** Kill an executor that is not running any receiver, if possible */ @@ -129,7 +135,9 @@ private[streaming] class ExecutorAllocationManager( if (allExecIds.nonEmpty && allExecIds.size > minNumExecutors) { val execIdsWithReceivers = receiverTracker.allocatedExecutors().values.flatten.toSeq - logInfo(s"Executors with receivers (${execIdsWithReceivers.size}): ${execIdsWithReceivers}") + logInfo(log"Executors with receivers (${MDC(LogKeys.EXECUTOR_IDS, + execIdsWithReceivers.size)}): " + + log"${MDC(LogKeys.EXECUTOR_IDS, execIdsWithReceivers)}") val removableExecIds = allExecIds.diff(execIdsWithReceivers) logDebug(s"Removable executors (${removableExecIds.size}): ${removableExecIds}") @@ -142,7 +150,7 @@ private[streaming] class ExecutorAllocationManager( } else { client.killExecutor(execIdToRemove) } - logInfo(s"Requested to kill executor $execIdToRemove") + logInfo(log"Requested to kill executor ${MDC(LogKeys.EXECUTOR_ID, execIdToRemove)}") } else { logInfo(s"No non-receiver executors to kill") } 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 639ac6de4f5d..bd9ea7b5a268 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 +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.streaming.{StreamingContext, Time} /** @@ -82,7 +82,8 @@ private[streaming] class InputInfoTracker(ssc: StreamingContext) extends Logging /** Cleanup the tracked input information older than threshold batch time */ def cleanup(batchThreshTime: Time): Unit = synchronized { val timesToCleanup = batchTimeToInputInfos.keys.filter(_ < batchThreshTime) - logInfo(s"remove old batch metadata: ${timesToCleanup.mkString(" ")}") + logInfo(log"remove old batch metadata: " + + log"${MDC(LogKeys.DURATION, timesToCleanup.mkString(" "))}") batchTimeToInputInfos --= timesToCleanup } } 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 f2700737384b..35c784060750 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, MDC} +import org.apache.spark.internal.{Logging, LogKeys, MDC} 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 @@ -127,7 +127,9 @@ private[streaming] class ReceivedBlockTracker( timeToAllocatedBlocks.put(batchTime, allocatedBlocks) lastAllocatedBatchTime = batchTime } else { - logInfo(s"Possibly processed batch $batchTime needs to be processed again in WAL recovery") + logInfo(log"Possibly processed batch ${MDC(LogKeys.BATCH_TIMESTAMP, + batchTime)} needs to be " + + log"processed again in WAL recovery") } } else { // This situation occurs when: @@ -137,7 +139,9 @@ private[streaming] class ReceivedBlockTracker( // 2. Slow checkpointing makes recovered batch time older than WAL recovered // lastAllocatedBatchTime. // This situation will only occurs in recovery time. - logInfo(s"Possibly processed batch $batchTime needs to be processed again in WAL recovery") + logInfo(log"Possibly processed batch ${MDC(LogKeys.BATCH_TIMESTAMP, + batchTime)} needs to be processed " + + log"again in WAL recovery") } } @@ -175,7 +179,7 @@ private[streaming] class ReceivedBlockTracker( def cleanupOldBatches(cleanupThreshTime: Time, waitForCompletion: Boolean): Unit = synchronized { require(cleanupThreshTime.milliseconds < clock.getTimeMillis()) val timesToCleanup = timeToAllocatedBlocks.keys.filter { _ < cleanupThreshTime }.toSeq - logInfo(s"Deleting batches: ${timesToCleanup.mkString(" ")}") + logInfo(log"Deleting batches: ${MDC(LogKeys.DURATION, timesToCleanup.mkString(" "))}") if (writeToLog(BatchCleanupEvent(timesToCleanup))) { timeToAllocatedBlocks --= timesToCleanup writeAheadLogOption.foreach(_.clean(cleanupThreshTime.milliseconds, waitForCompletion)) @@ -221,9 +225,10 @@ private[streaming] class ReceivedBlockTracker( } writeAheadLogOption.foreach { writeAheadLog => - logInfo(s"Recovering from write ahead logs in ${checkpointDirOption.get}") + logInfo(log"Recovering from write ahead logs in " + + log"${MDC(LogKeys.CHECKPOINT_DIR, checkpointDirOption.get)}") writeAheadLog.readAll().asScala.foreach { byteBuffer => - logInfo("Recovering record " + byteBuffer) + logInfo(log"Recovering record ${MDC(LogKeys.BYTE_BUFFER, byteBuffer)}") Utils.deserialize[ReceivedBlockTrackerLogEvent]( JavaUtils.bufferToArray(byteBuffer), Thread.currentThread().getContextClassLoader) match { case BlockAdditionEvent(receivedBlockInfo) => 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 48273b3b593c..f1fad343db02 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, MDC} +import org.apache.spark.internal.{Logging, LogKeys, MDC} 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._ @@ -232,7 +232,8 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false // Signal the receivers to delete old block data if (WriteAheadLogUtils.enableReceiverLog(ssc.conf)) { - logInfo(s"Cleanup old received batch data: $cleanupThreshTime") + logInfo(log"Cleanup old received batch data: " + + log"${MDC(LogKeys.CLEANUP_LOCAL_DIRS, cleanupThreshTime)}") synchronized { if (isTrackerStarted) { endpoint.send(CleanupOldBlocks(cleanupThreshTime)) @@ -306,7 +307,8 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false endpoint = Some(receiverEndpoint)) receiverTrackingInfos.put(streamId, receiverTrackingInfo) listenerBus.post(StreamingListenerReceiverStarted(receiverTrackingInfo.toReceiverInfo)) - logInfo("Registered receiver for stream " + streamId + " from " + senderAddress) + logInfo(log"Registered receiver for stream ${MDC(LogKeys.STREAM_ID, streamId)} " + + log"from ${MDC(LogKeys.SENDER_ADDRESS, senderAddress)}") true } } @@ -447,7 +449,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false runDummySparkJob() - logInfo("Starting " + receivers.length + " receivers") + logInfo(log"Starting ${MDC(LogKeys.NUM_RECEIVERS, receivers.length)} receivers") endpoint.send(StartAllReceivers(receivers.toImmutableArraySeq)) } @@ -625,7 +627,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false if (!shouldStartReceiver) { onReceiverJobFinish(receiverId) } else { - logInfo(s"Restarting Receiver $receiverId") + logInfo(log"Restarting Receiver ${MDC(LogKeys.RECEIVER_ID, receiverId)}") self.send(RestartReceiver(receiver)) } case Failure(e) => @@ -633,11 +635,11 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false onReceiverJobFinish(receiverId) } else { logError("Receiver has been stopped. Try to restart it.", e) - logInfo(s"Restarting Receiver $receiverId") + logInfo(log"Restarting Receiver ${MDC(LogKeys.RECEIVER_ID, receiverId)}") self.send(RestartReceiver(receiver)) } }(ThreadUtils.sameThread) - logInfo(s"Receiver ${receiver.streamId} started") + logInfo(log"Receiver ${MDC(LogKeys.RECEIVER_ID, receiver.streamId)} started") } override def onStop(): Unit = { @@ -660,7 +662,8 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false /** Send stop signal to the receivers. */ private def stopReceivers(): Unit = { receiverTrackingInfos.values.flatMap(_.endpoint).foreach { _.send(StopReceiver) } - logInfo("Sent stop signal to all " + receiverTrackingInfos.size + " receivers") + logInfo(log"Sent stop signal to all " + + log"${MDC(LogKeys.NUM_RECEIVERS, receiverTrackingInfos.size)} receivers") } } 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 dc02062b9eb4..1b05a6ac30cc 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 +import org.apache.spark.internal.{Logging, LogKeys, MDC} /** * Implements a proportional-integral-derivative (PID) controller which acts on @@ -74,8 +74,11 @@ private[streaming] class PIDRateEstimator( minRate > 0, s"Minimum rate in PIDRateEstimator should be > 0") - logInfo(s"Created PIDRateEstimator with proportional = $proportional, integral = $integral, " + - s"derivative = $derivative, min rate = $minRate") + logInfo(log"Created PIDRateEstimator with proportional = " + + log"${MDC(LogKeys.PROPORTIONAL, proportional)}, integral = " + + log"${MDC(LogKeys.INTEGRAL, integral)}, derivative = " + + log"${MDC(LogKeys.DERIVATIVE, derivative)}, min rate = " + + log"${MDC(LogKeys.MIN_RATE, minRate)}") def compute( time: Long, // in milliseconds 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 5dcdd573c744..8befe53efffa 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, MDC} +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.internal.LogKeys.RECORDS import org.apache.spark.network.util.JavaUtils import org.apache.spark.util.{ThreadUtils, Utils} @@ -122,7 +122,8 @@ private[util] class BatchedWriteAheadLog(val wrappedLog: WriteAheadLog, conf: Sp * Stop the batched writer thread, fulfill promises with failures and close the wrapped WAL. */ override def close(): Unit = { - logInfo(s"BatchedWriteAheadLog shutting down at time: ${System.currentTimeMillis()}.") + logInfo(log"BatchedWriteAheadLog shutting down at time: " + + log"${MDC(LogKeys.TIME, System.currentTimeMillis())}.") if (!active.getAndSet(false)) return batchedWriterThread.interrupt() batchedWriterThread.join() 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 58a6b929a81f..6ee5dd9025cf 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, MDC} +import org.apache.spark.internal.{Logging, LogKeys, MDC} 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._ @@ -137,7 +137,8 @@ private[streaming] class FileBasedWriteAheadLog( */ def readAll(): JIterator[ByteBuffer] = synchronized { val logFilesToRead = pastLogs.map{ _.path} ++ currentLogPath - logInfo("Reading from the logs:\n" + logFilesToRead.mkString("\n")) + logInfo(log"Reading from the logs:\n" + + log"${MDC(LogKeys.LOG_FILES, logFilesToRead.mkString("\n"))}") def readFile(file: String): Iterator[ByteBuffer] = { logDebug(s"Creating log reader with $file") val reader = new FileBasedWriteAheadLogReader(file, hadoopConf) @@ -170,8 +171,11 @@ private[streaming] class FileBasedWriteAheadLog( pastLogs --= expiredLogs expiredLogs } - logInfo(s"Attempting to clear ${oldLogFiles.size} old log files in $logDirectory " + - s"older than $threshTime: ${oldLogFiles.map { _.path }.mkString("\n")}") + logInfo(log"Attempting to clear ${MDC(LogKeys.NUM_RECORDS_READ, oldLogFiles.size)} " + + log"old log files in " + + log"${MDC(LogKeys.DIRECTORY, logDirectory)} older than " + + log"${MDC(LogKeys.THRESHOLD, threshTime)}: " + + log"${MDC(LogKeys.FILES, oldLogFiles.map(_.path).mkString("\n"))}") def deleteFile(walInfo: LogInfo): Unit = { try { @@ -184,7 +188,8 @@ private[streaming] class FileBasedWriteAheadLog( logWarning(log"Error clearing write ahead log file " + log"${MDC(WRITE_AHEAD_LOG_INFO, walInfo)}", ex) } - logInfo(s"Cleared log files in $logDirectory older than $threshTime") + logInfo(log"Cleared log files in ${MDC(LogKeys.LOG_DIRECTORY, logDirectory)} older than " + + log"${MDC(LogKeys.THRESH_TIME, threshTime)}") } oldLogFiles.foreach { logInfo => if (!executionContext.isShutdown) { @@ -252,7 +257,9 @@ private[streaming] class FileBasedWriteAheadLog( fileSystem.listStatus(logDirectoryPath).map { _.getPath }.toImmutableArraySeq) pastLogs.clear() pastLogs ++= logFileInfo - logInfo(s"Recovered ${logFileInfo.size} write ahead log files from $logDirectory") + logInfo(log"Recovered ${MDC(LogKeys.NUM_FILES, logFileInfo.size)} " + + log"write ahead log files from " + + log"${MDC(LogKeys.LOG_DIRECTORY, logDirectory)}") logDebug(s"Recovered files are:\n${logFileInfo.map(_.path).mkString("\n")}") } } catch { From b570a01ca0be7d9319e63e5563676501f0bc0e2d Mon Sep 17 00:00:00 2001 From: Amanda Liu Date: Thu, 13 Jun 2024 13:04:31 -0700 Subject: [PATCH 02/18] add LogKeys imports --- .../apache/spark/broadcast/Broadcast.scala | 2 +- .../spark/deploy/FaultToleranceTest.scala | 453 ++++++++++++++++++ .../shuffle/IndexShuffleBlockResolver.scala | 2 +- .../adaptive/ShufflePartitionsUtil.scala | 2 +- .../python/PythonStreamingSourceRunner.scala | 2 +- .../continuous/ContinuousExecution.scala | 4 +- 6 files changed, 459 insertions(+), 6 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala 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 672bef88c1cc..231024ba9556 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 +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.util.Utils /** diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala new file mode 100644 index 000000000000..d6a50ff84f56 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -0,0 +1,453 @@ +/* + * 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.deploy + +import java.io._ +import java.net.URL +import java.nio.charset.StandardCharsets +import java.util.concurrent.TimeoutException + +import scala.collection.mutable.ListBuffer +import scala.concurrent.{Future, Promise} +// scalastyle:off executioncontextglobal +import scala.concurrent.ExecutionContext.Implicits.global +// scalastyle:on executioncontextglobal +import scala.concurrent.duration._ +import scala.sys.process._ + +import org.json4s.Formats +import org.json4s.jackson.JsonMethods + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.deploy.master.RecoveryState +import org.apache.spark.internal.{config, Logging} +import org.apache.spark.util.{ThreadUtils, Utils} + +/** + * This suite tests the fault tolerance of the Spark standalone scheduler, mainly the Master. + * In order to mimic a real distributed cluster more closely, Docker is used. + * Execute using + * ./bin/spark-class org.apache.spark.deploy.FaultToleranceTest + * + * Make sure that the environment includes the following properties in SPARK_DAEMON_JAVA_OPTS: + * - spark.deploy.recoveryMode=ZOOKEEPER + * - spark.deploy.zookeeper.url=172.17.42.1:2181 + * Note that 172.17.42.1 is the default docker ip for the host and 2181 is the default ZK port. + * + * In case of failure, make sure to kill off prior docker containers before restarting: + * docker kill $(docker ps -q) + * + * Unfortunately, due to the Docker dependency this suite cannot be run automatically without a + * working installation of Docker. In addition to having Docker, the following are assumed: + * - Docker can run without sudo (see http://docs.docker.io/en/latest/use/basics/) + * - The docker images tagged spark-test-master and spark-test-worker are built from the + * docker/ directory. Run 'docker/spark-test/build' to generate these. + */ +private object FaultToleranceTest extends App with Logging { + + private val conf = new SparkConf() + private val zkDir = conf.get(config.Deploy.ZOOKEEPER_DIRECTORY).getOrElse("/spark") + + private val masters = ListBuffer[TestMasterInfo]() + private val workers = ListBuffer[TestWorkerInfo]() + private var sc: SparkContext = _ + + private val zk = SparkCuratorUtil.newClient(conf) + + private var numPassed = 0 + private var numFailed = 0 + + private val sparkHome = System.getenv("SPARK_HOME") + assertTrue(sparkHome != null, "Run with a valid SPARK_HOME") + + private val containerSparkHome = "/opt/spark" + private val dockerMountDir = "%s:%s".format(sparkHome, containerSparkHome) + + System.setProperty(config.DRIVER_HOST_ADDRESS.key, "172.17.42.1") // default docker host ip + + private def afterEach(): Unit = { + if (sc != null) { + sc.stop() + sc = null + } + terminateCluster() + + // Clear ZK directories in between tests (for speed purposes) + SparkCuratorUtil.deleteRecursive(zk, zkDir + "/spark_leader") + SparkCuratorUtil.deleteRecursive(zk, zkDir + "/master_status") + } + + test("sanity-basic") { + addMasters(1) + addWorkers(1) + createClient() + assertValidClusterState() + } + + test("sanity-many-masters") { + addMasters(3) + addWorkers(3) + createClient() + assertValidClusterState() + } + + test("single-master-halt") { + addMasters(3) + addWorkers(2) + createClient() + assertValidClusterState() + + killLeader() + delay(30.seconds) + assertValidClusterState() + createClient() + assertValidClusterState() + } + + test("single-master-restart") { + addMasters(1) + addWorkers(2) + createClient() + assertValidClusterState() + + killLeader() + addMasters(1) + delay(30.seconds) + assertValidClusterState() + + killLeader() + addMasters(1) + delay(30.seconds) + assertValidClusterState() + } + + test("cluster-failure") { + addMasters(2) + addWorkers(2) + createClient() + assertValidClusterState() + + terminateCluster() + addMasters(2) + addWorkers(2) + assertValidClusterState() + } + + test("all-but-standby-failure") { + addMasters(2) + addWorkers(2) + createClient() + assertValidClusterState() + + killLeader() + workers.foreach(_.kill()) + workers.clear() + delay(30.seconds) + addWorkers(2) + assertValidClusterState() + } + + test("rolling-outage") { + addMasters(1) + delay() + addMasters(1) + delay() + addMasters(1) + addWorkers(2) + createClient() + assertValidClusterState() + assertTrue(getLeader == masters.head) + + (1 to 3).foreach { _ => + killLeader() + delay(30.seconds) + assertValidClusterState() + assertTrue(getLeader == masters.head) + addMasters(1) + } + } + + private def test(name: String)(fn: => Unit): Unit = { + try { + fn + numPassed += 1 + logInfo("==============================================") + logInfo("Passed: " + name) + logInfo("==============================================") + } catch { + case e: Exception => + numFailed += 1 + logInfo("!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!") + logError("FAILED: " + name, e) + logInfo("!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!") + sys.exit(1) + } + afterEach() + } + + private def addMasters(num: Int): Unit = { + logInfo(s">>>>> ADD MASTERS $num <<<<<") + (1 to num).foreach { _ => masters += SparkDocker.startMaster(dockerMountDir) } + } + + private def addWorkers(num: Int): Unit = { + logInfo(s">>>>> ADD WORKERS $num <<<<<") + val masterUrls = getMasterUrls(masters.toSeq) + (1 to num).foreach { _ => workers += SparkDocker.startWorker(dockerMountDir, masterUrls) } + } + + /** Creates a SparkContext, which constructs a Client to interact with our cluster. */ + private def createClient() = { + logInfo(">>>>> CREATE CLIENT <<<<<") + if (sc != null) { sc.stop() } + // Counter-hack: Because of a hack in SparkEnv#create() that changes this + // property, we need to reset it. + System.setProperty(config.DRIVER_PORT.key, "0") + sc = new SparkContext(getMasterUrls(masters.toSeq), "fault-tolerance", containerSparkHome) + } + + private def getMasterUrls(masters: Seq[TestMasterInfo]): String = { + "spark://" + masters.map(master => master.ip + ":7077").mkString(",") + } + + private def getLeader: TestMasterInfo = { + val leaders = masters.filter(_.state == RecoveryState.ALIVE) + assertTrue(leaders.size == 1) + leaders(0) + } + + private def killLeader(): Unit = { + logInfo(">>>>> KILL LEADER <<<<<") + masters.foreach(_.readState()) + val leader = getLeader + masters -= leader + leader.kill() + } + + private def delay(secs: Duration = 5.seconds) = Thread.sleep(secs.toMillis) + + private def terminateCluster(): Unit = { + logInfo(">>>>> TERMINATE CLUSTER <<<<<") + masters.foreach(_.kill()) + workers.foreach(_.kill()) + masters.clear() + workers.clear() + } + + /** This includes Client retry logic, so it may take a while if the cluster is recovering. */ + private def assertUsable() = { + val f = Future { + try { + val res = sc.parallelize(0 until 10).collect() + assertTrue(res.toList == (0 until 10).toList) + true + } catch { + case e: Exception => + logError("assertUsable() had exception", e) + e.printStackTrace() + false + } + } + + // Avoid waiting indefinitely (e.g., we could register but get no executors). + assertTrue(ThreadUtils.awaitResult(f, 2.minutes)) + } + + /** + * Asserts that the cluster is usable and that the expected masters and workers + * are all alive in a proper configuration (e.g., only one leader). + */ + private def assertValidClusterState() = { + logInfo(">>>>> ASSERT VALID CLUSTER STATE <<<<<") + assertUsable() + var numAlive = 0 + var numStandby = 0 + var numLiveApps = 0 + var liveWorkerIPs: Seq[String] = List() + + def stateValid(): Boolean = { + workers.map(_.ip).forall(liveWorkerIPs.contains) && + numAlive == 1 && numStandby == masters.size - 1 && numLiveApps >= 1 + } + + val f = Future { + try { + while (!stateValid()) { + Thread.sleep(1000) + + numAlive = 0 + numStandby = 0 + numLiveApps = 0 + + masters.foreach(_.readState()) + + for (master <- masters) { + master.state match { + case RecoveryState.ALIVE => + numAlive += 1 + liveWorkerIPs = master.liveWorkerIPs + case RecoveryState.STANDBY => + numStandby += 1 + case _ => // ignore + } + + numLiveApps += master.numLiveApps + } + } + true + } catch { + case e: Exception => + logError("assertValidClusterState() had exception", e) + false + } + } + + try { + assertTrue(ThreadUtils.awaitResult(f, 2.minutes)) + } catch { + case e: TimeoutException => + logError("Master states: " + masters.map(_.state)) + logError("Num apps: " + numLiveApps) + logError("IPs expected: " + workers.map(_.ip) + " / found: " + liveWorkerIPs) + throw new RuntimeException("Failed to get into acceptable cluster state after 2 min.", e) + } + } + + private def assertTrue(bool: Boolean, message: String = ""): Unit = { + if (!bool) { + throw new IllegalStateException("Assertion failed: " + message) + } + } + + logInfo("Ran %s tests, %s passed and %s failed".format(numPassed + numFailed, numPassed, + numFailed)) +} + +private class TestMasterInfo(val ip: String, val dockerId: DockerId, val logFile: File) + extends Logging { + + implicit val formats: Formats = org.json4s.DefaultFormats + var state: RecoveryState.Value = _ + var liveWorkerIPs: List[String] = _ + var numLiveApps = 0 + + logDebug("Created master: " + this) + + def readState(): Unit = { + try { + val masterStream = new InputStreamReader( + new URL("http://%s:8080/json".format(ip)).openStream, StandardCharsets.UTF_8) + val json = JsonMethods.parse(masterStream) + + val workers = json \ "workers" + val liveWorkers = workers.children.filter(w => (w \ "state").extract[String] == "ALIVE") + // Extract the worker IP from "webuiaddress" (rather than "host") because the host name + // on containers is a weird hash instead of the actual IP address. + liveWorkerIPs = liveWorkers.map { + w => (w \ "webuiaddress").extract[String].stripPrefix("http://").stripSuffix(":8081") + } + + numLiveApps = (json \ "activeapps").children.size + + val status = json \\ "status" + val stateString = status.extract[String] + state = RecoveryState.values.filter(state => state.toString == stateString).head + } catch { + case e: Exception => + // ignore, no state update + logWarning("Exception", e) + } + } + + def kill(): Unit = { Docker.kill(dockerId) } + + override def toString: String = + "[ip=%s, id=%s, logFile=%s, state=%s]". + format(ip, dockerId.id, logFile.getAbsolutePath, state) +} + +private class TestWorkerInfo(val ip: String, val dockerId: DockerId, val logFile: File) + extends Logging { + + implicit val formats: Formats = org.json4s.DefaultFormats + + logDebug("Created worker: " + this) + + def kill(): Unit = { Docker.kill(dockerId) } + + override def toString: String = + "[ip=%s, id=%s, logFile=%s]".format(ip, dockerId, logFile.getAbsolutePath) +} + +private object SparkDocker { + def startMaster(mountDir: String): TestMasterInfo = { + val cmd = Docker.makeRunCmd("spark-test-master", mountDir = mountDir) + val (ip, id, outFile) = startNode(cmd) + new TestMasterInfo(ip, id, outFile) + } + + def startWorker(mountDir: String, masters: String): TestWorkerInfo = { + val cmd = Docker.makeRunCmd("spark-test-worker", args = masters, mountDir = mountDir) + val (ip, id, outFile) = startNode(cmd) + new TestWorkerInfo(ip, id, outFile) + } + + private def startNode(dockerCmd: ProcessBuilder) : (String, DockerId, File) = { + val ipPromise = Promise[String]() + val outFile = File.createTempFile("fault-tolerance-test", "", Utils.createTempDir()) + val outStream: FileWriter = new FileWriter(outFile) + def findIpAndLog(line: String): Unit = { + if (line.startsWith("CONTAINER_IP=")) { + val ip = line.split("=")(1) + ipPromise.success(ip) + } + + outStream.write(line + "\n") + outStream.flush() + } + + dockerCmd.run(ProcessLogger(findIpAndLog _)) + val ip = ThreadUtils.awaitResult(ipPromise.future, 30.seconds) + val dockerId = Docker.getLastProcessId + (ip, dockerId, outFile) + } +} + +private class DockerId(val id: String) { + override def toString: String = id +} + +private object Docker extends Logging { + def makeRunCmd(imageTag: String, args: String = "", mountDir: String = ""): ProcessBuilder = { + val mountCmd = if (mountDir != "") { " -v " + mountDir } else "" + + val cmd = "docker run -privileged %s %s %s".format(mountCmd, imageTag, args) + logDebug("Run command: " + cmd) + cmd + } + + def kill(dockerId: DockerId) : Unit = { + "docker kill %s".format(dockerId.id).! + } + + def getLastProcessId: DockerId = { + var id: String = null + "docker ps -l -q".!(ProcessLogger(line => id = line)) + new DockerId(id) + } +} 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 022a9c2ba9a8..e8cad955e4a2 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -28,7 +28,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, MDC} +import org.apache.spark.internal.{config, Logging, LogKeys, MDC} import org.apache.spark.internal.LogKeys._ import org.apache.spark.io.NioBufferedFileInputStream import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} 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 dbb8deb24400..47cc60ab404b 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 +import org.apache.spark.internal.{Logging, LogKeys, MDC} 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/python/PythonStreamingSourceRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala index 9d76cb063518..33612b6947f2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala @@ -27,7 +27,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, MDC} +import org.apache.spark.internal.{Logging, LogKeys, MDC} 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/streaming/continuous/ContinuousExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala index a56606f8c1cb..633aaf2682db 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 @@ -445,8 +445,8 @@ class ContinuousExecution( */ def stopInNewThread(error: Throwable): Unit = { if (failure.compareAndSet(null, error)) { - logError(log"Query ${MDC(LogKeys.PRETTY_ID_STRING, prettyIdString)} received exception " + - log"${MDC(LogKeys.ERROR, error)}") + logError(log"Query ${MDC(PRETTY_ID_STRING, prettyIdString)} received exception " + + log"${MDC(ERROR, error)}") stopInNewThread() } } From 280b2200337a6d3a2af22b79680f3e05dd649f55 Mon Sep 17 00:00:00 2001 From: Amanda Liu Date: Fri, 14 Jun 2024 07:37:22 -0700 Subject: [PATCH 03/18] update new logkey names --- .../scala/org/apache/spark/internal/LogKey.scala | 4 ++-- .../scala/org/apache/spark/util/MavenUtils.scala | 4 +++- .../SparkConnectStreamingQueryListenerHandler.scala | 6 +++--- .../scala/org/apache/spark/broadcast/Broadcast.scala | 2 +- .../apache/spark/deploy/ExternalShuffleService.scala | 4 ++-- .../spark/shuffle/IndexShuffleBlockResolver.scala | 2 +- .../spark/ml/classification/LogisticRegression.scala | 12 ++++++------ .../apache/spark/deploy/yarn/ApplicationMaster.scala | 2 +- .../scala/org/apache/spark/deploy/yarn/Client.scala | 7 +++---- scalastyle-config.xml | 8 -------- 10 files changed, 22 insertions(+), 29 deletions(-) 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 index 2ed1ac6d66ec..730f0c210c90 100644 --- a/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala +++ b/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala @@ -199,6 +199,7 @@ object LogKeys { case object DESIRED_NUM_PARTITIONS extends LogKey case object DESIRED_TREE_DEPTH extends LogKey case object DESTINATION_PATH extends LogKey + case object DESTROY_SITE extends LogKey case object DFS_FILE extends LogKey case object DIFF_DELTA extends LogKey case object DIRECTORY extends LogKey @@ -545,7 +546,7 @@ object LogKeys { case object OLD_VALUE extends LogKey case object OPEN_COST_IN_BYTES extends LogKey case object OPERATION_HANDLE extends LogKey - case object OPERATION_HANDLE_IDENTIFIER 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 @@ -828,7 +829,6 @@ object LogKeys { case object VOCAB_SIZE extends LogKey case object WAIT_RESULT_TIME extends LogKey case object WAIT_SEND_TIME extends LogKey - case object WAIT_TIME extends LogKey case object WATERMARK_CONSTRAINT extends LogKey case object WEB_URL extends LogKey case object WEIGHT extends LogKey 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 641af47e85ae..1bf3bbb76c4b 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 @@ -215,7 +215,9 @@ private[spark] object MavenUtils extends Logging { if (artifactInfo.getExt == "jar") { true } else { - logInfo(log"Skipping non-jar dependency ${MDC(LogKeys.ARTIFACT_ID, artifactInfo.getId)}") + // scalastyle:off + logInfo(s"Skipping non-jar dependency ${artifactInfo.getId}") + // scalastyle:on false } } diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala index de4bdf3d4451..fb21c637b70a 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala @@ -85,7 +85,7 @@ class SparkConnectStreamingQueryListenerHandler(executeHolder: ExecuteHolder) ex logError( log"[SessionId: ${MDC(LogKeys.SESSION_ID, sessionId)}]" + log"[UserId: ${MDC(LogKeys.USER_ID, userId)}]" + - log"[operationId: ${MDC(LogKeys.OPERATION_HANDLE_IDENTIFIER, + log"[operationId: ${MDC(LogKeys.OPERATION_ID, executeHolder.operationId)}] " + log"Error sending listener added response.", e) @@ -95,14 +95,14 @@ class SparkConnectStreamingQueryListenerHandler(executeHolder: ExecuteHolder) ex } logInfo(log"[SessionId: ${MDC(LogKeys.SESSION_ID, sessionId)}][UserId: " + log"${MDC(LogKeys.USER_ID, userId)}][operationId: " + - log"${MDC(LogKeys.OPERATION_HANDLE_IDENTIFIER, executeHolder.operationId)}] " + + log"${MDC(LogKeys.OPERATION_ID, executeHolder.operationId)}] " + log"Server side listener added. Now blocking until all client side listeners are " + log"removed or there is error transmitting the event back.") // Block the handling thread, and have serverListener continuously send back new events listenerHolder.streamingQueryListenerLatch.await() logInfo(log"[SessionId: ${MDC(LogKeys.SESSION_ID, sessionId)}][UserId: " + log"${MDC(LogKeys.USER_ID, userId)}]" + - log"[operationId: ${MDC(LogKeys.OPERATION_HANDLE, executeHolder.operationId)}] " + + log"[operationId: ${MDC(LogKeys.OPERATION_ID, executeHolder.operationId)}] " + log"Server side listener long-running handling thread ended.") case StreamingQueryListenerBusCommand.CommandCase.REMOVE_LISTENER_BUS_LISTENER => listenerHolder.isServerSideListenerRegistered match { 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 231024ba9556..f476d40237ad 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala @@ -106,7 +106,7 @@ abstract class Broadcast[T: ClassTag](val id: Long) extends Serializable with Lo assertValid() _isValid = false _destroySite = Utils.getCallSite().shortForm - logInfo(log"Destroying ${MDC(LogKeys.TO_STRING, toString)} " + + logInfo(log"Destroying ${MDC(LogKeys.BROADCAST, toString)} " + log"(from ${MDC(LogKeys.DESTROY_SITE, _destroySite)})") doDestroy(blocking) } 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 f7ec118d7160..851fb453fd09 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala @@ -71,8 +71,8 @@ class ExternalShuffleService(sparkConf: SparkConf, securityManager: SecurityMana if (localDirs.length >= 1) { new File(localDirs.find(new File(_, dbName).exists()).getOrElse(localDirs(0)), dbName) } else { - logWarning(log"'spark.local.dir' should be set first when we use db in " + - log"ExternalShuffleService. Note that this only affects standalone mode.") + logWarning("'spark.local.dir' should be set first when we use db in " + + "ExternalShuffleService. Note that this only affects standalone mode.") null } } 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 e8cad955e4a2..dde9b541b62f 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -437,7 +437,7 @@ private[spark] class IndexShuffleBlockResolver( try { if (!checksumTmp.delete()) { logError(log"Failed to delete temporary checksum file at " + - log"${MDC(LogKeys.FILE_ABSOLUTE_PATH, checksumTmp.getAbsolutePath)}") + log"${MDC(LogKeys.PATH, checksumTmp.getAbsolutePath)}") } } catch { case e: Exception => 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 8b9d7e802555..b3c48f13591f 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 @@ -503,8 +503,8 @@ class LogisticRegression @Since("1.2.0") ( tol, fitIntercept, maxBlockSizeInMB) if (dataset.storageLevel != StorageLevel.NONE) { - instr.logWarning(log"Input instances will be standardized, blockified to blocks, and " + - log"then cached during training. Be careful of double caching!") + instr.logWarning("Input instances will be standardized, blockified to blocks, and " + + "then cached during training. Be careful of double caching!") } val instances = dataset.select( @@ -569,8 +569,8 @@ class LogisticRegression @Since("1.2.0") ( val isConstantLabel = histogram.count(_ != 0.0) == 1 if ($(fitIntercept) && isConstantLabel && !usingBoundConstrainedOptimization) { - instr.logWarning(log"All labels are the same value and fitIntercept=true, so the " + - log"coefficients will be zeros. Training is not needed.") + instr.logWarning("All labels are the same value and fitIntercept=true, so the " + + "coefficients will be zeros. Training is not needed.") val constantLabelIndex = Vectors.dense(histogram).argmax val coefMatrix = new SparseMatrix(numCoefficientSets, numFeatures, new Array[Int](numCoefficientSets + 1), Array.emptyIntArray, Array.emptyDoubleArray, @@ -584,8 +584,8 @@ class LogisticRegression @Since("1.2.0") ( } if (!$(fitIntercept) && isConstantLabel) { - instr.logWarning(log"All labels belong to a single class and fitIntercept=false. It's a " + - log"dangerous ground, so the algorithm may not converge.") + instr.logWarning("All labels belong to a single class and fitIntercept=false. It's a " + + "dangerous ground, so the algorithm may not converge.") } val featuresMean = summarizer.mean.toArray 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 3bec02d12b1c..2523941e0bff 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 @@ -529,7 +529,7 @@ private[spark] class ApplicationMaster( case e: SparkException if e.getCause().isInstanceOf[TimeoutException] => logError( log"""SparkContext did not initialize after waiting for - |${MDC(LogKeys.WAIT_TIME, totalWaitTime)} ms. + |${MDC(LogKeys.TIMEOUT, totalWaitTime)} ms. | Please check earlier log output for errors. Failing the application.""".stripMargin) finish(FinalApplicationStatus.FAILED, ApplicationMaster.EXIT_SC_NOT_INITED, 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 c5673fdcd2be..bf31e03ba9a8 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 @@ -620,9 +620,8 @@ private[spark] class Client( // If we passed in a keytab, make sure we copy the keytab to the staging directory on // HDFS, and setup the relevant environment vars, so the AM can login again. amKeytabFileName.foreach { kt => - logInfo(log"To enable the AM to login from keytab, " + - log"credentials are being copied over to the AM " + - log"via the YARN Secure Distributed Cache.") + logInfo("To enable the AM to login from keytab, credentials are being copied over to the AM" + + " via the YARN Secure Distributed Cache.") val (_, localizedPath) = distribute(keytab, destName = Some(kt), appMasterOnly = true) @@ -1386,7 +1385,7 @@ private[spark] class Client( val YarnAppReport(appState, finalState, diags) = monitorApplication() if (appState == YarnApplicationState.FAILED || finalState == FinalApplicationStatus.FAILED) { diags.foreach { err => - logError(log"Application diagnostics message: ${MDC(LogKeys.MESSAGE, err)}") + logError(log"Application diagnostics message: ${MDC(LogKeys.ERROR, err)}") } throw new SparkException(s"Application $appId finished with failed status") } diff --git a/scalastyle-config.xml b/scalastyle-config.xml index 30192b8e3925..cd5a576c086f 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -155,14 +155,6 @@ This file is divided into 3 sections: - - log(?:Info|Warning|Error)\((?:".*"\.format\(.*\)|s".*(?:\$|\+\s*[^\s"]).*"\)) - - Logging message should use log"..." instead of s"..." and variables should be wrapped in `MDC`s. - Refer to Structured Logging Framework guidelines in the file `internal/Logging.scala`. - - - spark(.sqlContext)?.sparkContext.hadoopConfiguration Date: Fri, 14 Jun 2024 09:16:27 -0700 Subject: [PATCH 04/18] add mdc imports --- .../utils/src/main/scala/org/apache/spark/internal/LogKey.scala | 2 ++ .../scala/org/apache/spark/streaming/StreamingContext.scala | 2 +- .../spark/streaming/receiver/ReceiverSupervisorImpl.scala | 2 +- 3 files changed, 4 insertions(+), 2 deletions(-) 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 index 730f0c210c90..4b198572b818 100644 --- a/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala +++ b/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala @@ -282,6 +282,7 @@ object LogKeys { case object FINAL_OUTPUT_PATH extends LogKey case object FINAL_PATH extends LogKey case object FINISH_TRIGGER_DURATION extends LogKey + case object FORMATTED_CODE extends LogKey case object FREE_MEMORY_SIZE extends LogKey case object FROM_OFFSET extends LogKey case object FROM_TIME extends LogKey @@ -381,6 +382,7 @@ object LogKeys { 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_LINES extends LogKey case object MAX_MEMORY_SIZE extends LogKey case object MAX_METHOD_CODE_SIZE extends LogKey case object MAX_NUM_BINS extends LogKey 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 3898d00c7b06..24990db53292 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 +import org.apache.spark.internal.{Logging, LogKeys, MDC} 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/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index 9a7ea7db8c48..d7d3293f9629 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, MDC} +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.internal.LogKeys.{ERROR, MESSAGE} import org.apache.spark.rpc.{RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.storage.StreamBlockId From b65d9a675d705e60c9e8af96f1414753a94e7017 Mon Sep 17 00:00:00 2001 From: Amanda Liu Date: Mon, 17 Jun 2024 07:33:28 -0700 Subject: [PATCH 05/18] modify logkey names --- .../main/scala/org/apache/spark/internal/LogKey.scala | 11 +++++------ .../main/scala/org/apache/spark/util/MavenUtils.scala | 9 +++------ .../scala/org/apache/spark/broadcast/Broadcast.scala | 2 +- .../org/apache/spark/sql/types/UDTRegistration.scala | 2 +- .../catalyst/expressions/codegen/CodeGenerator.scala | 5 ++--- .../execution/adaptive/ShufflePartitionsUtil.scala | 2 +- .../sql/execution/streaming/StreamExecution.scala | 2 +- .../thriftserver/SparkExecuteStatementOperation.scala | 1 - .../org/apache/spark/streaming/StreamingContext.scala | 2 +- .../org/apache/spark/streaming/dstream/DStream.scala | 2 +- .../streaming/receiver/ReceiverSupervisorImpl.scala | 2 +- .../scheduler/ExecutorAllocationManager.scala | 4 ++-- .../spark/streaming/scheduler/ReceiverTracker.scala | 2 +- 13 files changed, 20 insertions(+), 26 deletions(-) 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 index 4b198572b818..d03068250e09 100644 --- a/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala +++ b/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala @@ -65,6 +65,7 @@ object LogKeys { 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 @@ -199,7 +200,6 @@ object LogKeys { case object DESIRED_NUM_PARTITIONS extends LogKey case object DESIRED_TREE_DEPTH extends LogKey case object DESTINATION_PATH extends LogKey - case object DESTROY_SITE extends LogKey case object DFS_FILE extends LogKey case object DIFF_DELTA extends LogKey case object DIRECTORY extends LogKey @@ -282,7 +282,6 @@ object LogKeys { case object FINAL_OUTPUT_PATH extends LogKey case object FINAL_PATH extends LogKey case object FINISH_TRIGGER_DURATION extends LogKey - case object FORMATTED_CODE extends LogKey case object FREE_MEMORY_SIZE extends LogKey case object FROM_OFFSET extends LogKey case object FROM_TIME extends LogKey @@ -569,6 +568,7 @@ object LogKeys { case object PARTITION_ID extends LogKey case object PARTITION_IDS extends LogKey case object PARTITION_SPECIFICATION extends LogKey + case object PARTITION_SIZE extends LogKey case object PARTITION_SPECS extends LogKey case object PATH extends LogKey case object PATHS extends LogKey @@ -617,6 +617,7 @@ object LogKeys { 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_CHECKPOINT_DIR extends LogKey case object RDD_DEBUG_STRING extends LogKey @@ -671,12 +672,10 @@ object LogKeys { case object SCALA_VERSION extends LogKey case object SCALING_UP_RATIO extends LogKey case object SCALING_DOWN_RATIO extends LogKey - case object SCALING_INTERVAL_SECS 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 SENDER_ADDRESS extends LogKey case object SERVER_NAME extends LogKey case object SERVICE_NAME extends LogKey case object SERVLET_CONTEXT_HANDLER_PATH extends LogKey @@ -754,9 +753,9 @@ object LogKeys { 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_LOCALITY extends LogKey @@ -785,6 +784,7 @@ object LogKeys { case object TIMEOUT extends LogKey case object TIMER extends LogKey case object TIMESTAMP extends LogKey + case object TIME_DURATION extends LogKey case object TIME_UNITS extends LogKey case object TIP extends LogKey case object TOKEN extends LogKey @@ -821,7 +821,6 @@ object LogKeys { case object URL extends LogKey case object URL2 extends LogKey case object URLS extends LogKey - case object USER_CLASS extends LogKey case object USER_ID extends LogKey case object USER_NAME extends LogKey case object UUID extends LogKey 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 1bf3bbb76c4b..ae00987cd69f 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 @@ -36,7 +36,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 import org.apache.spark.util.ArrayImplicits._ /** Provides utility functions to be used inside SparkSubmit. */ @@ -215,9 +215,7 @@ private[spark] object MavenUtils extends Logging { if (artifactInfo.getExt == "jar") { true } else { - // scalastyle:off logInfo(s"Skipping non-jar dependency ${artifactInfo.getId}") - // scalastyle:on false } } @@ -517,9 +515,8 @@ private[spark] object MavenUtils extends Logging { val failedReports = rr.getArtifactsReports(DownloadStatus.FAILED, true) if (failedReports.nonEmpty && noCacheIvySettings.isDefined) { val failedArtifacts = failedReports.map(r => r.getArtifact) - logInfo(log"Download failed: ${MDC(LogKeys.FAILED_ARTIFACTS, - failedArtifacts.mkString("[", ", ", "]"))}, " + - log"attempt to retry while skipping local-m2-cache.") + logInfo(s"Download failed: ${failedArtifacts.mkString("[", ", ", "]")}, " + + s"attempt to retry while skipping local-m2-cache.") failedArtifacts.foreach(artifact => { clearInvalidIvyCacheFiles(artifact.getModuleRevisionId, ivySettings.getDefaultCache) }) 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 f476d40237ad..3adb540a7ad1 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala @@ -107,7 +107,7 @@ abstract class Broadcast[T: ClassTag](val id: Long) extends Serializable with Lo _isValid = false _destroySite = Utils.getCallSite().shortForm logInfo(log"Destroying ${MDC(LogKeys.BROADCAST, toString)} " + - log"(from ${MDC(LogKeys.DESTROY_SITE, _destroySite)})") + log"(from ${MDC(LogKeys.CALL_SITE_SHORT_FORM, _destroySite)})") doDestroy(blocking) } 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 d7f07d5831c1..9219c1d139b9 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 @@ -58,7 +58,7 @@ object UDTRegistration extends Serializable with Logging { */ def register(userClass: String, udtClass: String): Unit = { if (udtMap.contains(userClass)) { - logWarning(log"Cannot register UDT for ${MDC(LogKeys.USER_CLASS, userClass)}, " + + logWarning(log"Cannot register UDT for ${MDC(LogKeys.CLASS_NAME, userClass)}, " + log"which is already registered.") } else { // When register UDT with class name, we can't check if the UDT class is an UserDefinedType, 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 b04d87c00554..a39c10866984 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 @@ -1561,10 +1561,9 @@ object CodeGenerator extends Logging { private def logGeneratedCode(code: CodeAndComment): Unit = { val maxLines = SQLConf.get.loggingMaxLinesForCodegen if (Utils.isTesting) { - logError(log"\n${MDC(LogKeys.FORMATTED_CODE, - CodeFormatter.format(code, MDC(LogKeys.MAX_LINES, maxLines)))}") + logError(s"\n${CodeFormatter.format(code, maxLines)}") } else { - logInfo(log"\n${MDC(LogKeys.FORMATTED_CODE, CodeFormatter.format(code, maxLines))}") + logInfo(s"\n${CodeFormatter.format(code, maxLines)}") } } 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 47cc60ab404b..bb7d904402de 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 @@ -62,7 +62,7 @@ object ShufflePartitionsUtil extends Logging { val shuffleIds = mapOutputStatistics.flatMap(_.map(_.shuffleId)).mkString(", ") logInfo(log"For shuffle(${MDC(LogKeys.SHUFFLE_ID, shuffleIds)}, advisory target size: " + - log"${MDC(LogKeys.TARGET_SIZE, advisoryTargetSize)}, actual target size " + + log"${MDC(LogKeys.ADVISORY_TARGET_SIZE, advisoryTargetSize)}, actual target size " + log"${MDC(LogKeys.TARGET_SIZE, targetSize)}, minimum partition size: " + log"${MDC(LogKeys.PARTITION_SIZE, minPartitionSize)}") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 77c08ce9bf6f..4198d7367fe2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -323,7 +323,7 @@ abstract class StreamExecution( // Log logical plan at the start of the query to help debug issues related to // plan changes. logInfo(log"Finish initializing with logical plan:\n" + - log"${MDC(LogKeys.LOGICAL_PLAN_LEAVES, logicalPlan)}") + log"${MDC(LogKeys.QUERY_PLAN, logicalPlan)}") // Unblock `awaitInitialization` initializationLatch.countDown() 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 51a5e88aa633..a61219021437 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 @@ -31,7 +31,6 @@ 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.LogKeys._ import org.apache.spark.sql.{DataFrame, Row, SQLContext} import org.apache.spark.sql.catalyst.util.CharVarcharUtils import org.apache.spark.sql.catalyst.util.DateTimeConstants.MILLIS_PER_SECOND 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 24990db53292..94b695e6452e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -726,7 +726,7 @@ class StreamingContext private[streaming] ( private def stopOnShutdown(): Unit = { val stopGracefully = conf.get(STOP_GRACEFULLY_ON_SHUTDOWN) logInfo(log"Invoking stop(stopGracefully=" + - log"${MDC(LogKeys.STOP_SITE_SHORT_FORM, stopGracefully)}) from shutdown hook") + log"${MDC(LogKeys.VALUE, stopGracefully)}) from shutdown hook") // Do not stop SparkContext, let its own shutdown hook stop it stop(stopSparkContext = false, stopGracefully = stopGracefully) } 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 9b86d8b6df82..552d7de726a3 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 @@ -320,7 +320,7 @@ abstract class DStream[T: ClassTag] ( logInfo(log"Time ${MDC(LogKeys.TIME, time)} is invalid as zeroTime is " + log"${MDC(LogKeys.ZERO_TIME, zeroTime)}, slideDuration is " + log"${MDC(LogKeys.SLIDE_DURATION, slideDuration)} and difference is " + - log"${MDC(LogKeys.TIME, time - zeroTime)}") + log"${MDC(LogKeys.TIME_DURATION, time - zeroTime)}") false } else { logDebug(s"Time $time is valid") 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 d7d3293f9629..aafa99bd5285 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 @@ -198,7 +198,7 @@ private[streaming] class ReceiverSupervisorImpl( logInfo(log"Deregistering receiver ${MDC(LogKeys.STREAM_ID, streamId)}") val errorString = error.map(Throwables.getStackTraceAsString).getOrElse("") trackerEndpoint.askSync[Boolean](DeregisterReceiver(streamId, message, errorString)) - logInfo(log"Stopped receiver ${MDC(LogKeys.RECEIVER_ID, streamId)}") + logInfo(log"Stopped receiver ${MDC(LogKeys.STREAM_ID, streamId)}") } override def createBlockGenerator( 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 00750df0b8d0..054db48181cd 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 @@ -78,7 +78,7 @@ private[streaming] class ExecutorAllocationManager( logInfo(log"ExecutorAllocationManager started with ratios = " + log"[${MDC(LogKeys.SCALING_UP_RATIO, scalingUpRatio)}, " + log"${MDC(LogKeys.SCALING_DOWN_RATIO, scalingDownRatio)}] and interval = " + - log"${MDC(LogKeys.SCALING_INTERVAL_SECS, scalingIntervalSecs)} sec") + log"${MDC(LogKeys.INTERVAL, scalingIntervalSecs)} sec") } def stop(): Unit = { @@ -124,7 +124,7 @@ private[streaming] class ExecutorAllocationManager( Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> targetTotalExecutors), Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> 0), Map.empty) - logInfo(log"Requested total ${MDC(LogKeys.TARGET_NUM_EXECUTOR, + logInfo(log"Requested total ${MDC(LogKeys.NUM_EXECUTORS, targetTotalExecutors)} executors") } 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 f1fad343db02..2604aebd6434 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 @@ -308,7 +308,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false receiverTrackingInfos.put(streamId, receiverTrackingInfo) listenerBus.post(StreamingListenerReceiverStarted(receiverTrackingInfo.toReceiverInfo)) logInfo(log"Registered receiver for stream ${MDC(LogKeys.STREAM_ID, streamId)} " + - log"from ${MDC(LogKeys.SENDER_ADDRESS, senderAddress)}") + log"from ${MDC(LogKeys.RPC_ADDRESS, senderAddress)}") true } } From 9f06627f7224efec2ae6cd028e7475ba209278a5 Mon Sep 17 00:00:00 2001 From: Amanda Liu Date: Mon, 17 Jun 2024 10:39:03 -0700 Subject: [PATCH 06/18] add TARGET_NUM_EXECUTOR log key --- .../utils/src/main/scala/org/apache/spark/internal/LogKey.scala | 1 + 1 file changed, 1 insertion(+) 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 index d03068250e09..cd348fcd5f4e 100644 --- a/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala +++ b/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala @@ -753,6 +753,7 @@ object LogKeys { 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 From 9c368ddc9011d6fdd817a83015d9dd19b8cc1886 Mon Sep 17 00:00:00 2001 From: Amanda Liu Date: Mon, 17 Jun 2024 12:01:28 -0700 Subject: [PATCH 07/18] add sink log key --- .../utils/src/main/scala/org/apache/spark/internal/LogKey.scala | 1 + 1 file changed, 1 insertion(+) 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 index 38a35f7deca6..c83c857f38e4 100644 --- a/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala +++ b/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala @@ -697,6 +697,7 @@ private[spark] object LogKeys { 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 From 85049f395398f9d4f9e8ddcd97dfc9499e545119 Mon Sep 17 00:00:00 2001 From: Amanda Liu Date: Mon, 17 Jun 2024 13:19:04 -0700 Subject: [PATCH 08/18] update log key names --- .../src/main/scala/org/apache/spark/internal/LogKey.scala | 1 - .../apache/spark/streaming/scheduler/ReceiverTracker.scala | 6 +++--- .../spark/streaming/util/FileBasedWriteAheadLog.scala | 4 ++-- 3 files changed, 5 insertions(+), 6 deletions(-) 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 index c83c857f38e4..169d5630efa0 100644 --- a/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala +++ b/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala @@ -362,7 +362,6 @@ private[spark] object LogKeys { case object LOCATION extends LogKey case object LOGICAL_PLAN_COLUMNS extends LogKey case object LOGICAL_PLAN_LEAVES extends LogKey - case object LOG_DIRECTORY extends LogKey case object LOG_FILES extends LogKey case object LOG_ID extends LogKey case object LOG_KEY_FILE extends LogKey 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 2604aebd6434..a37ba04c1012 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 @@ -627,7 +627,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false if (!shouldStartReceiver) { onReceiverJobFinish(receiverId) } else { - logInfo(log"Restarting Receiver ${MDC(LogKeys.RECEIVER_ID, receiverId)}") + logInfo(log"Restarting Receiver ${MDC(LogKeys.STREAM_ID, receiverId)}") self.send(RestartReceiver(receiver)) } case Failure(e) => @@ -635,11 +635,11 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false onReceiverJobFinish(receiverId) } else { logError("Receiver has been stopped. Try to restart it.", e) - logInfo(log"Restarting Receiver ${MDC(LogKeys.RECEIVER_ID, receiverId)}") + logInfo(log"Restarting Receiver ${MDC(LogKeys.STREAM_ID, receiverId)}") self.send(RestartReceiver(receiver)) } }(ThreadUtils.sameThread) - logInfo(log"Receiver ${MDC(LogKeys.RECEIVER_ID, receiver.streamId)} started") + logInfo(log"Receiver ${MDC(LogKeys.STREAM_ID, receiver.streamId)} started") } override def onStop(): Unit = { 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 6ee5dd9025cf..8b9368c5960b 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 @@ -188,7 +188,7 @@ private[streaming] class FileBasedWriteAheadLog( logWarning(log"Error clearing write ahead log file " + log"${MDC(WRITE_AHEAD_LOG_INFO, walInfo)}", ex) } - logInfo(log"Cleared log files in ${MDC(LogKeys.LOG_DIRECTORY, logDirectory)} older than " + + logInfo(log"Cleared log files in ${MDC(LogKeys.PATH, logDirectory)} older than " + log"${MDC(LogKeys.THRESH_TIME, threshTime)}") } oldLogFiles.foreach { logInfo => @@ -259,7 +259,7 @@ private[streaming] class FileBasedWriteAheadLog( pastLogs ++= logFileInfo logInfo(log"Recovered ${MDC(LogKeys.NUM_FILES, logFileInfo.size)} " + log"write ahead log files from " + - log"${MDC(LogKeys.LOG_DIRECTORY, logDirectory)}") + log"${MDC(LogKeys.PATH, logDirectory)}") logDebug(s"Recovered files are:\n${logFileInfo.map(_.path).mkString("\n")}") } } catch { From e94f2804781e068820ce563b1d56ccf605c43b29 Mon Sep 17 00:00:00 2001 From: Amanda Liu Date: Mon, 17 Jun 2024 13:55:38 -0700 Subject: [PATCH 09/18] rename logkeys --- .../src/main/scala/org/apache/spark/internal/LogKey.scala | 8 +------- .../org/apache/spark/streaming/dstream/DStream.scala | 6 +++--- .../streaming/scheduler/ExecutorAllocationManager.scala | 2 +- .../spark/streaming/scheduler/ReceivedBlockTracker.scala | 2 +- .../spark/streaming/util/FileBasedWriteAheadLog.scala | 4 ++-- 5 files changed, 8 insertions(+), 14 deletions(-) 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 index 169d5630efa0..b90aba184d23 100644 --- a/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala +++ b/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala @@ -115,7 +115,6 @@ private[spark] object LogKeys { case object CANCEL_FUTURE_JOBS extends LogKey case object CATALOG_NAME extends LogKey case object CATEGORICAL_FEATURES extends LogKey - case object CHECKPOINT_DIR extends LogKey case object CHECKPOINT_FILE extends LogKey case object CHECKPOINT_INTERVAL extends LogKey case object CHECKPOINT_LOCATION extends LogKey @@ -202,7 +201,6 @@ private[spark] object LogKeys { case object DESTINATION_PATH extends LogKey case object DFS_FILE extends LogKey case object DIFF_DELTA extends LogKey - case object DIRECTORY extends LogKey case object DIVISIBLE_CLUSTER_INDICES_SIZE extends LogKey case object DRIVER_ID extends LogKey case object DRIVER_MEMORY_SIZE extends LogKey @@ -254,7 +252,6 @@ private[spark] object LogKeys { case object EXPR extends LogKey case object EXPR_TERMS extends LogKey case object EXTENDED_EXPLAIN_GENERATOR extends LogKey - case object FAILED_ARTIFACTS extends LogKey case object FAILED_STAGE extends LogKey case object FAILED_STAGE_NAME extends LogKey case object FAILURES extends LogKey @@ -362,7 +359,6 @@ private[spark] object LogKeys { case object LOCATION extends LogKey case object LOGICAL_PLAN_COLUMNS extends LogKey case object LOGICAL_PLAN_LEAVES extends LogKey - case object LOG_FILES extends LogKey case object LOG_ID extends LogKey case object LOG_KEY_FILE extends LogKey case object LOG_LEVEL extends LogKey @@ -380,7 +376,6 @@ private[spark] object LogKeys { 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_LINES extends LogKey case object MAX_MEMORY_SIZE extends LogKey case object MAX_METHOD_CODE_SIZE extends LogKey case object MAX_NUM_BINS extends LogKey @@ -735,6 +730,7 @@ private[spark] object LogKeys { case object STORAGE_LEVEL_REPLICATION extends LogKey case object STORAGE_MEMORY_SIZE extends LogKey case object STORE_ID 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 @@ -781,12 +777,10 @@ private[spark] object LogKeys { case object THREAD_POOL_WAIT_QUEUE_SIZE extends LogKey case object THRESHOLD extends LogKey case object THRESH_TIME extends LogKey - case object TID 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_DURATION extends LogKey case object TIME_UNITS extends LogKey case object TIP extends LogKey case object TOKEN extends LogKey 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 552d7de726a3..87d6a4909fdd 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 @@ -280,7 +280,7 @@ abstract class DStream[T: ClassTag] ( logInfo(log"Slide time = ${MDC(LogKeys.SLIDE_DURATION, slideDuration)}") logInfo(log"Storage level = ${MDC(LogKeys.STORAGE_LEVEL, storageLevel.description)}") - logInfo(log"Checkpoint interval = ${MDC(LogKeys.CHECKPOINT_INTERVAL, checkpointDuration)}") + logInfo(log"Checkpoint interval = ${MDC(LogKeys.CHECKPOINT_INTERVAL, checkpointDuration)}") logInfo(log"Remember interval = ${MDC(LogKeys.INTERVAL, rememberDuration)}") logInfo(log"Initialized and validated ${MDC(LogKeys.DSTREAM, this)}") } @@ -290,7 +290,7 @@ abstract class DStream[T: ClassTag] ( throw new SparkException(s"Context must not be set again for $this") } ssc = s - logInfo(log"Set context for ${MDC(LogKeys.CONTEXT, this)}") + logInfo(log"Set context for ${MDC(LogKeys.STREAMING_CONTEXT, this)}") dependencies.foreach(_.setContext(ssc)) } @@ -320,7 +320,7 @@ abstract class DStream[T: ClassTag] ( logInfo(log"Time ${MDC(LogKeys.TIME, time)} is invalid as zeroTime is " + log"${MDC(LogKeys.ZERO_TIME, zeroTime)}, slideDuration is " + log"${MDC(LogKeys.SLIDE_DURATION, slideDuration)} and difference is " + - log"${MDC(LogKeys.TIME_DURATION, time - zeroTime)}") + log"${MDC(LogKeys.DURATION, time - zeroTime)}") false } else { logDebug(s"Time $time is valid") 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 054db48181cd..903cde8082db 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 @@ -135,7 +135,7 @@ private[streaming] class ExecutorAllocationManager( if (allExecIds.nonEmpty && allExecIds.size > minNumExecutors) { val execIdsWithReceivers = receiverTracker.allocatedExecutors().values.flatten.toSeq - logInfo(log"Executors with receivers (${MDC(LogKeys.EXECUTOR_IDS, + logInfo(log"Executors with receivers (${MDC(LogKeys.NUM_EXECUTORS, execIdsWithReceivers.size)}): " + log"${MDC(LogKeys.EXECUTOR_IDS, execIdsWithReceivers)}") 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 35c784060750..7fb35a04be6d 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 @@ -226,7 +226,7 @@ private[streaming] class ReceivedBlockTracker( writeAheadLogOption.foreach { writeAheadLog => logInfo(log"Recovering from write ahead logs in " + - log"${MDC(LogKeys.CHECKPOINT_DIR, checkpointDirOption.get)}") + log"${MDC(LogKeys.PATH, checkpointDirOption.get)}") writeAheadLog.readAll().asScala.foreach { byteBuffer => logInfo(log"Recovering record ${MDC(LogKeys.BYTE_BUFFER, byteBuffer)}") Utils.deserialize[ReceivedBlockTrackerLogEvent]( 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 8b9368c5960b..d90095c73785 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 @@ -138,7 +138,7 @@ private[streaming] class FileBasedWriteAheadLog( def readAll(): JIterator[ByteBuffer] = synchronized { val logFilesToRead = pastLogs.map{ _.path} ++ currentLogPath logInfo(log"Reading from the logs:\n" + - log"${MDC(LogKeys.LOG_FILES, logFilesToRead.mkString("\n"))}") + log"${MDC(LogKeys.PATHS, logFilesToRead.mkString("\n"))}") def readFile(file: String): Iterator[ByteBuffer] = { logDebug(s"Creating log reader with $file") val reader = new FileBasedWriteAheadLogReader(file, hadoopConf) @@ -173,7 +173,7 @@ private[streaming] class FileBasedWriteAheadLog( } logInfo(log"Attempting to clear ${MDC(LogKeys.NUM_RECORDS_READ, oldLogFiles.size)} " + log"old log files in " + - log"${MDC(LogKeys.DIRECTORY, logDirectory)} older than " + + log"${MDC(LogKeys.PATH, logDirectory)} older than " + log"${MDC(LogKeys.THRESHOLD, threshTime)}: " + log"${MDC(LogKeys.FILES, oldLogFiles.map(_.path).mkString("\n"))}") From 945d0be9e2bd7e92176f6af10364f66116d0d5b4 Mon Sep 17 00:00:00 2001 From: Amanda Liu Date: Mon, 17 Jun 2024 14:07:44 -0700 Subject: [PATCH 10/18] keep OPERATION_HANDLE_IDENTIFIER --- .../src/main/scala/org/apache/spark/internal/LogKey.scala | 2 +- .../planner/SparkConnectStreamingQueryListenerHandler.scala | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) 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 index b90aba184d23..ae42ffc57676 100644 --- a/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala +++ b/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala @@ -541,7 +541,7 @@ private[spark] object LogKeys { case object OLD_VALUE extends LogKey case object OPEN_COST_IN_BYTES extends LogKey case object OPERATION_HANDLE extends LogKey - case object OPERATION_ID extends LogKey + case object OPERATION_HANDLE_IDENTIFIER extends LogKey case object OPTIMIZED_PLAN_COLUMNS extends LogKey case object OPTIMIZER_CLASS_NAME extends LogKey case object OPTIONS extends LogKey diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala index fb21c637b70a..200be7be6407 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala @@ -85,7 +85,7 @@ class SparkConnectStreamingQueryListenerHandler(executeHolder: ExecuteHolder) ex logError( log"[SessionId: ${MDC(LogKeys.SESSION_ID, sessionId)}]" + log"[UserId: ${MDC(LogKeys.USER_ID, userId)}]" + - log"[operationId: ${MDC(LogKeys.OPERATION_ID, + log"[operationId: ${MDC(LogKeys.OPERATION_HANDLE_IDENTIFIER, executeHolder.operationId)}] " + log"Error sending listener added response.", e) @@ -95,14 +95,14 @@ class SparkConnectStreamingQueryListenerHandler(executeHolder: ExecuteHolder) ex } logInfo(log"[SessionId: ${MDC(LogKeys.SESSION_ID, sessionId)}][UserId: " + log"${MDC(LogKeys.USER_ID, userId)}][operationId: " + - log"${MDC(LogKeys.OPERATION_ID, executeHolder.operationId)}] " + + log"${MDC(LogKeys.OPERATION_HANDLE_IDENTIFIER, executeHolder.operationId)}] " + log"Server side listener added. Now blocking until all client side listeners are " + log"removed or there is error transmitting the event back.") // Block the handling thread, and have serverListener continuously send back new events listenerHolder.streamingQueryListenerLatch.await() logInfo(log"[SessionId: ${MDC(LogKeys.SESSION_ID, sessionId)}][UserId: " + log"${MDC(LogKeys.USER_ID, userId)}]" + - log"[operationId: ${MDC(LogKeys.OPERATION_ID, executeHolder.operationId)}] " + + log"[operationId: ${MDC(LogKeys.OPERATION_HANDLE_IDENTIFIER, executeHolder.operationId)}] " + log"Server side listener long-running handling thread ended.") case StreamingQueryListenerBusCommand.CommandCase.REMOVE_LISTENER_BUS_LISTENER => listenerHolder.isServerSideListenerRegistered match { From 3c646f011789af3be385ba67b1d62ff26df19a0a Mon Sep 17 00:00:00 2001 From: Amanda Liu Date: Mon, 17 Jun 2024 15:30:39 -0700 Subject: [PATCH 11/18] lint fix --- .../planner/SparkConnectStreamingQueryListenerHandler.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala index 200be7be6407..8fd43f9e1ed3 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala @@ -102,7 +102,8 @@ class SparkConnectStreamingQueryListenerHandler(executeHolder: ExecuteHolder) ex listenerHolder.streamingQueryListenerLatch.await() logInfo(log"[SessionId: ${MDC(LogKeys.SESSION_ID, sessionId)}][UserId: " + log"${MDC(LogKeys.USER_ID, userId)}]" + - log"[operationId: ${MDC(LogKeys.OPERATION_HANDLE_IDENTIFIER, executeHolder.operationId)}] " + + log"[operationId: ${MDC(LogKeys.OPERATION_HANDLE_IDENTIFIER, + executeHolder.operationId)}] " + log"Server side listener long-running handling thread ended.") case StreamingQueryListenerBusCommand.CommandCase.REMOVE_LISTENER_BUS_LISTENER => listenerHolder.isServerSideListenerRegistered match { From 6e9d4436787961ac4ae1f130c3b96baf10fe8b72 Mon Sep 17 00:00:00 2001 From: Amanda Liu Date: Mon, 17 Jun 2024 17:56:42 -0700 Subject: [PATCH 12/18] scala format --- .../streaming/StreamingQueryListenerBus.scala | 6 ++++-- ...ConnectStreamingQueryListenerHandler.scala | 19 +++++++++---------- 2 files changed, 13 insertions(+), 12 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListenerBus.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListenerBus.scala index a3391c7c9fbc..c3c23740e2fe 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListenerBus.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListenerBus.scala @@ -144,8 +144,10 @@ class StreamingQueryListenerBus(sparkSession: SparkSession) extends Logging { listener.onQueryIdle(t) case t: QueryTerminatedEvent => listener.onQueryTerminated(t) - case _ => logWarning(log"Unknown StreamingQueryListener event: " + - log"${MDC(LogKeys.EVENT, event)}") + case _ => + logWarning( + log"Unknown StreamingQueryListener event: " + + log"${MDC(LogKeys.EVENT, event)}") } } catch { case e: Exception => diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala index 8fd43f9e1ed3..71798d37ee98 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala @@ -85,25 +85,24 @@ class SparkConnectStreamingQueryListenerHandler(executeHolder: ExecuteHolder) ex logError( log"[SessionId: ${MDC(LogKeys.SESSION_ID, sessionId)}]" + log"[UserId: ${MDC(LogKeys.USER_ID, userId)}]" + - log"[operationId: ${MDC(LogKeys.OPERATION_HANDLE_IDENTIFIER, - executeHolder.operationId)}] " + - log"Error sending listener added response.", + log"[operationId: ${MDC(LogKeys.OPERATION_HANDLE_IDENTIFIER, executeHolder.operationId)}] " + + log"Error sending listener added response.", e) listenerHolder.cleanUp() return } } - logInfo(log"[SessionId: ${MDC(LogKeys.SESSION_ID, sessionId)}][UserId: " + - log"${MDC(LogKeys.USER_ID, userId)}][operationId: " + - log"${MDC(LogKeys.OPERATION_HANDLE_IDENTIFIER, executeHolder.operationId)}] " + - log"Server side listener added. Now blocking until all client side listeners are " + - log"removed or there is error transmitting the event back.") + logInfo( + log"[SessionId: ${MDC(LogKeys.SESSION_ID, sessionId)}][UserId: " + + log"${MDC(LogKeys.USER_ID, userId)}][operationId: " + + log"${MDC(LogKeys.OPERATION_HANDLE_IDENTIFIER, executeHolder.operationId)}] " + + log"Server side listener added. Now blocking until all client side listeners are " + + log"removed or there is error transmitting the event back.") // Block the handling thread, and have serverListener continuously send back new events listenerHolder.streamingQueryListenerLatch.await() logInfo(log"[SessionId: ${MDC(LogKeys.SESSION_ID, sessionId)}][UserId: " + log"${MDC(LogKeys.USER_ID, userId)}]" + - log"[operationId: ${MDC(LogKeys.OPERATION_HANDLE_IDENTIFIER, - executeHolder.operationId)}] " + + log"[operationId: ${MDC(LogKeys.OPERATION_HANDLE_IDENTIFIER, executeHolder.operationId)}] " + log"Server side listener long-running handling thread ended.") case StreamingQueryListenerBusCommand.CommandCase.REMOVE_LISTENER_BUS_LISTENER => listenerHolder.isServerSideListenerRegistered match { From d1c26768859d1c284c876de2562c583b8d92a773 Mon Sep 17 00:00:00 2001 From: Amanda Liu Date: Tue, 18 Jun 2024 07:48:37 -0700 Subject: [PATCH 13/18] lint fix --- .../planner/SparkConnectStreamingQueryListenerHandler.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala index 71798d37ee98..59507d8c9141 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala @@ -85,7 +85,8 @@ class SparkConnectStreamingQueryListenerHandler(executeHolder: ExecuteHolder) ex logError( log"[SessionId: ${MDC(LogKeys.SESSION_ID, sessionId)}]" + log"[UserId: ${MDC(LogKeys.USER_ID, userId)}]" + - log"[operationId: ${MDC(LogKeys.OPERATION_HANDLE_IDENTIFIER, executeHolder.operationId)}] " + + log"[operationId: ${MDC(LogKeys.OPERATION_HANDLE_IDENTIFIER, + executeHolder.operationId)}] " + log"Error sending listener added response.", e) listenerHolder.cleanUp() @@ -102,7 +103,8 @@ class SparkConnectStreamingQueryListenerHandler(executeHolder: ExecuteHolder) ex listenerHolder.streamingQueryListenerLatch.await() logInfo(log"[SessionId: ${MDC(LogKeys.SESSION_ID, sessionId)}][UserId: " + log"${MDC(LogKeys.USER_ID, userId)}]" + - log"[operationId: ${MDC(LogKeys.OPERATION_HANDLE_IDENTIFIER, executeHolder.operationId)}] " + + log"[operationId: ${MDC(LogKeys.OPERATION_HANDLE_IDENTIFIER, + executeHolder.operationId)}] " + log"Server side listener long-running handling thread ended.") case StreamingQueryListenerBusCommand.CommandCase.REMOVE_LISTENER_BUS_LISTENER => listenerHolder.isServerSideListenerRegistered match { From f2a79b8d393fd19a4cb82010a03dc16214eb0380 Mon Sep 17 00:00:00 2001 From: Amanda Liu Date: Tue, 18 Jun 2024 09:34:43 -0700 Subject: [PATCH 14/18] scala format --- .../planner/SparkConnectStreamingQueryListenerHandler.scala | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala index 59507d8c9141..71798d37ee98 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala @@ -85,8 +85,7 @@ class SparkConnectStreamingQueryListenerHandler(executeHolder: ExecuteHolder) ex logError( log"[SessionId: ${MDC(LogKeys.SESSION_ID, sessionId)}]" + log"[UserId: ${MDC(LogKeys.USER_ID, userId)}]" + - log"[operationId: ${MDC(LogKeys.OPERATION_HANDLE_IDENTIFIER, - executeHolder.operationId)}] " + + log"[operationId: ${MDC(LogKeys.OPERATION_HANDLE_IDENTIFIER, executeHolder.operationId)}] " + log"Error sending listener added response.", e) listenerHolder.cleanUp() @@ -103,8 +102,7 @@ class SparkConnectStreamingQueryListenerHandler(executeHolder: ExecuteHolder) ex listenerHolder.streamingQueryListenerLatch.await() logInfo(log"[SessionId: ${MDC(LogKeys.SESSION_ID, sessionId)}][UserId: " + log"${MDC(LogKeys.USER_ID, userId)}]" + - log"[operationId: ${MDC(LogKeys.OPERATION_HANDLE_IDENTIFIER, - executeHolder.operationId)}] " + + log"[operationId: ${MDC(LogKeys.OPERATION_HANDLE_IDENTIFIER, executeHolder.operationId)}] " + log"Server side listener long-running handling thread ended.") case StreamingQueryListenerBusCommand.CommandCase.REMOVE_LISTENER_BUS_LISTENER => listenerHolder.isServerSideListenerRegistered match { From ebe7bec442b17dbe460bfc088545a73a014515c6 Mon Sep 17 00:00:00 2001 From: Amanda Liu Date: Tue, 18 Jun 2024 11:33:47 -0700 Subject: [PATCH 15/18] lint --- .../SparkConnectStreamingQueryListenerHandler.scala | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala index 71798d37ee98..d072b56e022a 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala @@ -85,7 +85,8 @@ class SparkConnectStreamingQueryListenerHandler(executeHolder: ExecuteHolder) ex logError( log"[SessionId: ${MDC(LogKeys.SESSION_ID, sessionId)}]" + log"[UserId: ${MDC(LogKeys.USER_ID, userId)}]" + - log"[operationId: ${MDC(LogKeys.OPERATION_HANDLE_IDENTIFIER, executeHolder.operationId)}] " + + log"[operationId: " + + log"${MDC(LogKeys.OPERATION_HANDLE_IDENTIFIER, executeHolder.operationId)}] " + log"Error sending listener added response.", e) listenerHolder.cleanUp() @@ -100,10 +101,12 @@ class SparkConnectStreamingQueryListenerHandler(executeHolder: ExecuteHolder) ex log"removed or there is error transmitting the event back.") // Block the handling thread, and have serverListener continuously send back new events listenerHolder.streamingQueryListenerLatch.await() - logInfo(log"[SessionId: ${MDC(LogKeys.SESSION_ID, sessionId)}][UserId: " + - log"${MDC(LogKeys.USER_ID, userId)}]" + - log"[operationId: ${MDC(LogKeys.OPERATION_HANDLE_IDENTIFIER, executeHolder.operationId)}] " + - log"Server side listener long-running handling thread ended.") + logInfo( + log"[SessionId: ${MDC(LogKeys.SESSION_ID, sessionId)}][UserId: " + + log"${MDC(LogKeys.USER_ID, userId)}]" + + log"[operationId: " + + log"${MDC(LogKeys.OPERATION_HANDLE_IDENTIFIER, executeHolder.operationId)}] " + + log"Server side listener long-running handling thread ended.") case StreamingQueryListenerBusCommand.CommandCase.REMOVE_LISTENER_BUS_LISTENER => listenerHolder.isServerSideListenerRegistered match { case true => From dd510b341b2c3b63c99abcbd91c4908c20017adc Mon Sep 17 00:00:00 2001 From: Amanda Liu Date: Tue, 18 Jun 2024 13:40:47 -0700 Subject: [PATCH 16/18] add logkeys imports --- .../scheduler/cluster/YarnClientSchedulerBackend.scala | 2 +- .../thriftserver/SparkExecuteStatementOperation.scala | 1 + .../spark/sql/hive/thriftserver/SparkSQLCLIService.scala | 8 +------- 3 files changed, 3 insertions(+), 8 deletions(-) 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 119657409899..8032d782cf4f 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, MDC} +import org.apache.spark.internal.{config, Logging, LogKeys, MDC} 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/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 a61219021437..51a5e88aa633 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 @@ -31,6 +31,7 @@ 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.LogKeys._ import org.apache.spark.sql.{DataFrame, Row, SQLContext} import org.apache.spark.sql.catalyst.util.CharVarcharUtils import org.apache.spark.sql.catalyst.util.DateTimeConstants.MILLIS_PER_SECOND diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala index 9421c8ae496d..b0c170903c0a 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala @@ -34,7 +34,7 @@ import org.apache.hive.service.auth.HiveAuthFactory import org.apache.hive.service.cli._ import org.apache.hive.service.server.HiveServer2 -import org.apache.spark.internal.{Logging, LogKeys, MDC, SparkLogger} +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.util.SQLKeywordUtils import org.apache.spark.sql.errors.QueryExecutionErrors @@ -112,12 +112,6 @@ private[hive] class SparkSQLCLIService(hiveServer: HiveServer2, sqlContext: SQLC } private[thriftserver] trait ReflectedCompositeService { this: AbstractService => - - private val logInfo = (msg: String) => getAncestorField[SparkLogger](this, 3, "LOG").info(msg) - - private val logError = (msg: String, e: Throwable) => - getAncestorField[SparkLogger](this, 3, "LOG").error(msg, e) - def initCompositeService(hiveConf: HiveConf): Unit = { // Emulating `CompositeService.init(hiveConf)` val serviceList = getAncestorField[JList[Service]](this, 2, "serviceList") From 12e068037b244abaabba0fa03eeeb56ca540f0e0 Mon Sep 17 00:00:00 2001 From: Amanda Liu Date: Tue, 18 Jun 2024 14:36:56 -0700 Subject: [PATCH 17/18] revert SparkSQLCLI SparkLogger use --- .../sql/hive/thriftserver/SparkSQLCLIService.scala | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala index b0c170903c0a..46537f75f1a1 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala @@ -34,7 +34,7 @@ import org.apache.hive.service.auth.HiveAuthFactory import org.apache.hive.service.cli._ import org.apache.hive.service.server.HiveServer2 -import org.apache.spark.internal.{Logging, LogKeys, MDC} +import org.apache.spark.internal.SparkLogger import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.util.SQLKeywordUtils import org.apache.spark.sql.errors.QueryExecutionErrors @@ -112,6 +112,12 @@ private[hive] class SparkSQLCLIService(hiveServer: HiveServer2, sqlContext: SQLC } private[thriftserver] trait ReflectedCompositeService { this: AbstractService => + + private val logInfo = (msg: String) => getAncestorField[SparkLogger](this, 3, "LOG").info(msg) + + private val logError = (msg: String, e: Throwable) => + getAncestorField[SparkLogger](this, 3, "LOG").error(msg, e) + def initCompositeService(hiveConf: HiveConf): Unit = { // Emulating `CompositeService.init(hiveConf)` val serviceList = getAncestorField[JList[Service]](this, 2, "serviceList") @@ -121,7 +127,7 @@ private[thriftserver] trait ReflectedCompositeService { this: AbstractService => invoke(classOf[AbstractService], this, "ensureCurrentState", classOf[STATE] -> STATE.NOTINITED) setAncestorField(this, 3, "hiveConf", hiveConf) invoke(classOf[AbstractService], this, "changeState", classOf[STATE] -> STATE.INITED) - logInfo(log"Service: ${MDC(LogKeys.SERVICE_NAME, getName)} is inited.") + logInfo(s"Service: $getName is inited.") } def startCompositeService(): Unit = { @@ -138,10 +144,10 @@ private[thriftserver] trait ReflectedCompositeService { this: AbstractService => setAncestorField(this, 3, "startTime", startTime) invoke(classOf[AbstractService], this, "ensureCurrentState", classOf[STATE] -> STATE.INITED) invoke(classOf[AbstractService], this, "changeState", classOf[STATE] -> STATE.STARTED) - logInfo(log"Service: ${MDC(LogKeys.SERVICE_NAME, getName)} is started.") + logInfo(s"Service: $getName is started.") } catch { case NonFatal(e) => - logError(log"Error starting services ${MDC(LogKeys.SERVICE_NAME, getName)}", e) + logError(s"Error starting services $getName", e) invoke(classOf[CompositeService], this, "stop", classOf[Int] -> Integer.valueOf(serviceStartCount)) throw HiveThriftServerErrors.failedToStartServiceError(getName, e) From b5e87090c62e14f60cd9c87b3bd5ab44ba87c242 Mon Sep 17 00:00:00 2001 From: Amanda Liu Date: Tue, 18 Jun 2024 17:57:08 -0700 Subject: [PATCH 18/18] sort log keys --- .../src/main/scala/org/apache/spark/internal/LogKey.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 index ae42ffc57676..f90eb4a77071 100644 --- a/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala +++ b/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala @@ -103,9 +103,9 @@ private[spark] object LogKeys { 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 BYTECODE_SIZE extends LogKey case object CACHED_TABLE_PARTITION_METADATA_SIZE extends LogKey case object CACHE_AUTO_REMOVED_SIZE extends LogKey case object CACHE_UNTIL_HIGHEST_CONSUMED_SIZE extends LogKey @@ -561,8 +561,8 @@ private[spark] object LogKeys { case object PARTITIONER extends LogKey case object PARTITION_ID extends LogKey case object PARTITION_IDS extends LogKey - case object PARTITION_SPECIFICATION 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 @@ -664,8 +664,8 @@ private[spark] object LogKeys { case object RULE_NAME extends LogKey case object RUN_ID extends LogKey case object SCALA_VERSION extends LogKey - case object SCALING_UP_RATIO 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