-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-47602][CORE] Resource managers: Migrate logError with variables to structured logging framework #45808
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
6edda2b
8ed977f
98669a2
bf70abb
23ce760
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -29,14 +29,15 @@ import org.apache.logging.log4j.core.filter.AbstractFilter | |
| import org.slf4j.{Logger, LoggerFactory} | ||
|
|
||
| import org.apache.spark.internal.Logging.SparkShellLoggingFilter | ||
| import org.apache.spark.internal.LogKey.LogKey | ||
| import org.apache.spark.util.SparkClassUtils | ||
|
|
||
| /** | ||
| * Mapped Diagnostic Context (MDC) that will be used in log messages. | ||
| * The values of the MDC will be inline in the log message, while the key-value pairs will be | ||
| * part of the ThreadContext. | ||
| */ | ||
| case class MDC(key: LogKey.Value, value: String) | ||
| case class MDC(key: LogKey, value: Any) | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Make the |
||
|
|
||
| /** | ||
| * Wrapper class for log messages that include a logging context. | ||
|
|
@@ -102,9 +103,9 @@ trait Logging { | |
| val context = new java.util.HashMap[String, String]() | ||
|
|
||
| args.foreach { mdc => | ||
| sb.append(mdc.value) | ||
| sb.append(mdc.value.toString) | ||
| if (Logging.isStructuredLoggingEnabled) { | ||
| context.put(mdc.key.toString.toLowerCase(Locale.ROOT), mdc.value) | ||
| context.put(mdc.key.toString.toLowerCase(Locale.ROOT), mdc.value.toString) | ||
| } | ||
|
|
||
| if (processedParts.hasNext) { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,49 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark.util | ||
|
|
||
| import scala.jdk.CollectionConverters._ | ||
|
|
||
| import org.scalatest.funsuite.AnyFunSuite // scalastyle:ignore funsuite | ||
|
|
||
| import org.apache.spark.internal.{Logging, MDC} | ||
| import org.apache.spark.internal.LogKey.EXIT_CODE | ||
|
|
||
| class MDCSuite | ||
| extends AnyFunSuite // scalastyle:ignore funsuite | ||
| with Logging { | ||
|
|
||
| test("check MDC message") { | ||
| val log = log"This is a log, exitcode ${MDC(EXIT_CODE, 10086)}" | ||
| assert(log.message === "This is a log, exitcode 10086") | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Let's verify the returned map as well
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Okay |
||
| assert(log.context === Map("exit_code" -> "10086").asJava) | ||
| } | ||
|
|
||
| test("custom object as MDC value") { | ||
| val cov = CustomObjectValue("spark", 10086) | ||
| val log = log"This is a log, exitcode ${MDC(EXIT_CODE, cov)}" | ||
| assert(log.message === "This is a log, exitcode CustomObjectValue: spark, 10086") | ||
| assert(log.context === Map("exit_code" -> "CustomObjectValue: spark, 10086").asJava) | ||
| } | ||
|
|
||
| case class CustomObjectValue(key: String, value: Int) { | ||
| override def toString: String = { | ||
| "CustomObjectValue: " + key + ", " + value | ||
| } | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -14,6 +14,7 @@ | |
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark.util | ||
|
|
||
| import java.io.File | ||
|
|
@@ -57,14 +58,20 @@ trait LoggingSuiteBase | |
|
|
||
| def msgWithMDCAndException: LogEntry = log"Error in executor ${MDC(EXECUTOR_ID, "1")}." | ||
|
|
||
| def expectedPatternForBasicMsg(level: Level): String | ||
|
|
||
| def msgWithConcat: LogEntry = log"Min Size: ${MDC(MIN_SIZE, "2")}, " + | ||
| log"Max Size: ${MDC(MAX_SIZE, "4")}. " + | ||
| log"Please double check." | ||
|
|
||
| // test for basic message (without any mdc) | ||
| def expectedPatternForBasicMsg(level: Level): String | ||
|
|
||
| // test for basic message and exception | ||
| def expectedPatternForBasicMsgWithException(level: Level): String | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Add a new
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Nit: Let's add comments about each pattern is about
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Okay |
||
|
|
||
| // test for message (with mdc) | ||
| def expectedPatternForMsgWithMDC(level: Level): String | ||
|
|
||
| // test for message and exception | ||
| def expectedPatternForMsgWithMDCAndException(level: Level): String | ||
|
|
||
| def verifyMsgWithConcat(level: Level, logOutput: String): Unit | ||
|
|
@@ -79,6 +86,17 @@ trait LoggingSuiteBase | |
| } | ||
| } | ||
|
|
||
| test("Basic logging with Exception") { | ||
| val exception = new RuntimeException("OOM") | ||
| Seq( | ||
| (Level.ERROR, () => logError(basicMsg, exception)), | ||
| (Level.WARN, () => logWarning(basicMsg, exception)), | ||
| (Level.INFO, () => logInfo(basicMsg, exception))).foreach { case (level, logFunc) => | ||
| val logOutput = captureLogOutput(logFunc) | ||
| assert(expectedPatternForBasicMsgWithException(level).r.matches(logOutput)) | ||
| } | ||
| } | ||
|
|
||
| test("Logging with MDC") { | ||
| Seq( | ||
| (Level.ERROR, () => logError(msgWithMDC)), | ||
|
|
@@ -98,7 +116,7 @@ trait LoggingSuiteBase | |
| (Level.INFO, () => logInfo(msgWithMDCAndException, exception))).foreach { | ||
| case (level, logFunc) => | ||
| val logOutput = captureLogOutput(logFunc) | ||
| assert(expectedPatternForMsgWithMDCAndException(level).r.findFirstIn(logOutput).isDefined) | ||
| assert(expectedPatternForMsgWithMDCAndException(level).r.matches(logOutput)) | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -137,6 +155,22 @@ class StructuredLoggingSuite extends LoggingSuiteBase { | |
| }""") | ||
| } | ||
|
|
||
| override def expectedPatternForBasicMsgWithException(level: Level): String = { | ||
| compactAndToRegexPattern( | ||
| s""" | ||
| { | ||
| "ts": "<timestamp>", | ||
| "level": "$level", | ||
| "msg": "This is a log message", | ||
| "exception": { | ||
| "class": "java.lang.RuntimeException", | ||
| "msg": "OOM", | ||
| "stacktrace": "<stacktrace>" | ||
| }, | ||
| "logger": "$className" | ||
| }""") | ||
| } | ||
|
|
||
| override def expectedPatternForMsgWithMDC(level: Level): String = { | ||
| compactAndToRegexPattern( | ||
| s""" | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -33,7 +33,7 @@ import org.apache.spark.deploy.k8s.Config._ | |
| import org.apache.spark.deploy.k8s.Constants._ | ||
| import org.apache.spark.deploy.k8s.KubernetesConf | ||
| import org.apache.spark.deploy.k8s.KubernetesUtils.addOwnerReference | ||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.internal.{Logging, LogKey, MDC} | ||
| import org.apache.spark.internal.config._ | ||
| import org.apache.spark.resource.ResourceProfile | ||
| import org.apache.spark.scheduler.cluster.SchedulerBackendUtils.DEFAULT_NUMBER_EXECUTORS | ||
|
|
@@ -143,7 +143,8 @@ class ExecutorPodsAllocator( | |
| snapshotsStore.addSubscriber(podAllocationDelay) { executorPodsSnapshot => | ||
| onNewSnapshots(applicationId, schedulerBackend, executorPodsSnapshot) | ||
| if (failureTracker.numFailedExecutors > maxNumExecutorFailures) { | ||
| logError(s"Max number of executor failures ($maxNumExecutorFailures) reached") | ||
| logError(log"Max number of executor failures " + | ||
| log"(${MDC(LogKey.MAX_EXECUTOR_FAILURES, maxNumExecutorFailures)}) reached") | ||
| stopApplication(EXCEED_MAX_EXECUTOR_FAILURES) | ||
| } | ||
| } | ||
|
|
@@ -532,7 +533,8 @@ class ExecutorPodsAllocator( | |
| currentTime - creationTime > executorIdleTimeout | ||
| } catch { | ||
| case e: Exception => | ||
| logError(s"Cannot get the creationTimestamp of the pod: ${state.pod}", e) | ||
| logError(log"Cannot get the creationTimestamp of the pod: " + | ||
| log"${MDC(LogKey.POD_ID, state.pod)}", e) | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. IMO
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @pan3793 Thanks for the suggestion. Do you mind creating a PR to improve this?
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I would like to have a try, it may take a few hours to learn this new log framework.
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The inital PR of the framework is in #45729. Thank you in advance!
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @pan3793 Thank you very much for |
||
| true | ||
| } | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -43,7 +43,8 @@ import org.apache.spark.deploy.{ExecutorFailureTracker, SparkHadoopUtil} | |
| import org.apache.spark.deploy.history.HistoryServer | ||
| import org.apache.spark.deploy.security.HadoopDelegationTokenManager | ||
| import org.apache.spark.deploy.yarn.config._ | ||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.internal.{Logging, MDC} | ||
| import org.apache.spark.internal.LogKey.{EXIT_CODE, REMOTE_ADDRESS} | ||
| import org.apache.spark.internal.config._ | ||
| import org.apache.spark.internal.config.UI._ | ||
| import org.apache.spark.metrics.{MetricsSystem, MetricsSystemInstances} | ||
|
|
@@ -745,9 +746,10 @@ private[spark] class ApplicationMaster( | |
| case _: InterruptedException => | ||
| // Reporter thread can interrupt to stop user class | ||
| case SparkUserAppException(exitCode) => | ||
| val msg = s"User application exited with status $exitCode" | ||
| val msg = log"User application exited with status " + | ||
| log"${MDC(EXIT_CODE, exitCode)}" | ||
| logError(msg) | ||
| finish(FinalApplicationStatus.FAILED, exitCode, msg) | ||
| finish(FinalApplicationStatus.FAILED, exitCode, msg.message) | ||
| case cause: Throwable => | ||
| logError("User class threw exception: ", cause) | ||
| finish(FinalApplicationStatus.FAILED, | ||
|
|
@@ -854,7 +856,8 @@ private[spark] class ApplicationMaster( | |
| logInfo(s"Driver terminated or disconnected! Shutting down. $remoteAddress") | ||
| finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS) | ||
| } else { | ||
| logError(s"Driver terminated with exit code ${exitCode}! Shutting down. $remoteAddress") | ||
| logError(log"Driver terminated with exit code ${MDC(EXIT_CODE, exitCode)}! " + | ||
| log"Shutting down. ${MDC(REMOTE_ADDRESS, remoteAddress)}") | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @gengliangwang
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @panbingkun Let's put all the variables into the context during the migration. If the context is too verbose, we can configure log4j to only show a subset of keys later.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Okay |
||
| finish(FinalApplicationStatus.FAILED, exitCode) | ||
| } | ||
| } else { | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
why do we need this?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Because I want to use 'xxx: LogKey' instead of 'xxx: LogKey.Value' to define
a type, it feels like it's apersonal preference. Should werestoreit?spark/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala
Lines 23 to 32 in c3da260