-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-47598][CORE] MLLib: Migrate logError with variables to structured logging framework #45837
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 1 commit
05127ba
cf22181
c46a601
7b4fe42
46d2fd6
5b4f2a1
7dfa115
27f8d05
01a8303
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 |
|---|---|---|
|
|
@@ -49,6 +49,8 @@ case class MessageWithContext(message: String, context: java.util.HashMap[String | |
| resultMap.putAll(mdc.context) | ||
| MessageWithContext(message + mdc.message, resultMap) | ||
| } | ||
|
|
||
| override def toString: String = message | ||
panbingkun marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| } | ||
|
|
||
| /** | ||
|
|
@@ -117,7 +119,7 @@ trait Logging { | |
| } | ||
| } | ||
|
|
||
| private def withLogContext(context: java.util.HashMap[String, String])(body: => Unit): Unit = { | ||
| protected def withLogContext(context: java.util.HashMap[String, String])(body: => Unit): Unit = { | ||
|
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. Why change |
||
| val threadContext = CloseableThreadContext.putAll(context) | ||
| try { | ||
| body | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -25,7 +25,8 @@ import org.apache.hadoop.fs.Path | |
|
|
||
| import org.apache.spark.SparkException | ||
| import org.apache.spark.annotation.Since | ||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.internal.{Logging, MDC} | ||
| import org.apache.spark.internal.LogKey.{NUM_CLASSIFICATION_LABELS, OPTIMIZER_CLASS_NAME, RANGE_CLASSIFICATION_LABELS} | ||
| import org.apache.spark.ml.feature._ | ||
| import org.apache.spark.ml.linalg._ | ||
| import org.apache.spark.ml.optim.aggregator._ | ||
|
|
@@ -220,10 +221,11 @@ class LinearSVC @Since("2.2.0") ( | |
| instr.logNumFeatures(numFeatures) | ||
|
|
||
| if (numInvalid != 0) { | ||
| val msg = s"Classification labels should be in [0 to ${numClasses - 1}]. " + | ||
| s"Found $numInvalid invalid labels." | ||
| val msg = log"Classification labels should be in " + | ||
| log"${MDC(RANGE_CLASSIFICATION_LABELS, s"[0 to ${numClasses - 1}]")}. " + | ||
|
||
| log"Found ${MDC(NUM_CLASSIFICATION_LABELS, numInvalid)} invalid labels." | ||
|
||
| instr.logError(msg) | ||
| throw new SparkException(msg) | ||
| throw new SparkException(msg.message) | ||
| } | ||
|
|
||
| val featuresStd = summarizer.std.toArray | ||
|
|
@@ -249,9 +251,9 @@ class LinearSVC @Since("2.2.0") ( | |
| regularization, optimizer) | ||
|
|
||
| if (rawCoefficients == null) { | ||
| val msg = s"${optimizer.getClass.getName} failed." | ||
| val msg = log"${MDC(OPTIMIZER_CLASS_NAME, optimizer.getClass.getName)} failed." | ||
| instr.logError(msg) | ||
| throw new SparkException(msg) | ||
| throw new SparkException(msg.message) | ||
| } | ||
|
|
||
| val coefficientArray = Array.tabulate(numFeatures) { i => | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -25,7 +25,8 @@ import org.apache.hadoop.fs.Path | |
|
|
||
| import org.apache.spark.SparkException | ||
| import org.apache.spark.annotation.Since | ||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.internal.{Logging, MDC} | ||
| import org.apache.spark.internal.LogKey.OPTIMIZER_CLASS_NAME | ||
| import org.apache.spark.ml.PredictorParams | ||
| import org.apache.spark.ml.feature._ | ||
| import org.apache.spark.ml.linalg._ | ||
|
|
@@ -271,9 +272,9 @@ class AFTSurvivalRegression @Since("1.6.0") (@Since("1.6.0") override val uid: S | |
| optimizer, initialSolution) | ||
|
|
||
| if (rawCoefficients == null) { | ||
| val msg = s"${optimizer.getClass.getName} failed." | ||
| val msg = log"${MDC(OPTIMIZER_CLASS_NAME, optimizer.getClass.getName)} failed." | ||
|
||
| instr.logError(msg) | ||
| throw new SparkException(msg) | ||
| throw new SparkException(msg.message) | ||
| } | ||
|
|
||
| val coefficientArray = Array.tabulate(numFeatures) { i => | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -27,7 +27,7 @@ import org.json4s._ | |
| import org.json4s.JsonDSL._ | ||
| import org.json4s.jackson.JsonMethods._ | ||
|
|
||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.internal.{LogEntry, Logging, MessageWithContext} | ||
| import org.apache.spark.ml.{MLEvents, PipelineStage} | ||
| import org.apache.spark.ml.param.{Param, Params} | ||
| import org.apache.spark.rdd.RDD | ||
|
|
@@ -84,20 +84,53 @@ private[spark] class Instrumentation private () extends Logging with MLEvents { | |
| super.logWarning(prefix + msg) | ||
| } | ||
|
|
||
| /** | ||
| * Logs a LogEntry which message with a prefix that uniquely identifies the training session. | ||
| */ | ||
| override def logWarning(entry: LogEntry): Unit = { | ||
| if (log.isWarnEnabled) { | ||
| withLogContext(entry.context) { | ||
| log.warn(prefix + entry.message) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Logs a error message with a prefix that uniquely identifies the training session. | ||
| */ | ||
| override def logError(msg: => String): Unit = { | ||
| super.logError(prefix + msg) | ||
| } | ||
|
|
||
| /** | ||
| * Logs a LogEntry which message with a prefix that uniquely identifies the training session. | ||
| */ | ||
| override def logError(entry: LogEntry): Unit = { | ||
|
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. Maybe we can write it as follows: But it seems that the |
||
| if (log.isErrorEnabled) { | ||
| withLogContext(entry.context) { | ||
| log.error(prefix + entry.message) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Logs an info message with a prefix that uniquely identifies the training session. | ||
| */ | ||
| override def logInfo(msg: => String): Unit = { | ||
| super.logInfo(prefix + msg) | ||
| } | ||
|
|
||
| /** | ||
| * Logs a LogEntry which message with a prefix that uniquely identifies the training session. | ||
| */ | ||
| override def logInfo(entry: LogEntry): Unit = { | ||
| if (log.isInfoEnabled) { | ||
| withLogContext(entry.context) { | ||
| log.info(prefix + entry.message) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Logs the value of the given parameters for the estimator being used in this session. | ||
| */ | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,6 +18,8 @@ | |
| package org.apache.spark.mllib.tree | ||
|
|
||
| import org.apache.spark.SparkFunSuite | ||
| import org.apache.spark.internal.LogKey.{LEARNING_RATE, NUM_ITERATIONS, SUBSAMPLING_RATE} | ||
| import org.apache.spark.internal.MDC | ||
| import org.apache.spark.mllib.regression.LabeledPoint | ||
| import org.apache.spark.mllib.tree.configuration.{BoostingStrategy, Strategy} | ||
| import org.apache.spark.mllib.tree.configuration.Algo._ | ||
|
|
@@ -51,8 +53,9 @@ class GradientBoostedTreesSuite extends SparkFunSuite with MLlibTestSparkContext | |
| gbt, GradientBoostedTreesSuite.data.toImmutableArraySeq, 0.06) | ||
| } catch { | ||
| case e: java.lang.AssertionError => | ||
| logError(s"FAILED for numIterations=$numIterations, learningRate=$learningRate," + | ||
| s" subsamplingRate=$subsamplingRate") | ||
| logError(log"FAILED for numIterations=${MDC(NUM_ITERATIONS, numIterations)}, " + | ||
|
||
| log"learningRate=${MDC(LEARNING_RATE, learningRate)}, " + | ||
| log"subsamplingRate=${MDC(SUBSAMPLING_RATE, subsamplingRate)}") | ||
| throw e | ||
| } | ||
|
|
||
|
|
@@ -82,8 +85,9 @@ class GradientBoostedTreesSuite extends SparkFunSuite with MLlibTestSparkContext | |
| gbt, GradientBoostedTreesSuite.data.toImmutableArraySeq, 0.85, "mae") | ||
| } catch { | ||
| case e: java.lang.AssertionError => | ||
| logError(s"FAILED for numIterations=$numIterations, learningRate=$learningRate," + | ||
| s" subsamplingRate=$subsamplingRate") | ||
| logError(log"FAILED for numIterations=${MDC(NUM_ITERATIONS, numIterations)}, " + | ||
| log"learningRate=${MDC(LEARNING_RATE, learningRate)}, " + | ||
| log"subsamplingRate=${MDC(SUBSAMPLING_RATE, subsamplingRate)}") | ||
| throw e | ||
| } | ||
|
|
||
|
|
@@ -114,8 +118,9 @@ class GradientBoostedTreesSuite extends SparkFunSuite with MLlibTestSparkContext | |
| gbt, GradientBoostedTreesSuite.data.toImmutableArraySeq, 0.9) | ||
| } catch { | ||
| case e: java.lang.AssertionError => | ||
| logError(s"FAILED for numIterations=$numIterations, learningRate=$learningRate," + | ||
| s" subsamplingRate=$subsamplingRate") | ||
| logError(log"FAILED for numIterations=${MDC(NUM_ITERATIONS, numIterations)}, " + | ||
| log"learningRate=${MDC(LEARNING_RATE, learningRate)}, " + | ||
| log"subsamplingRate=${MDC(SUBSAMPLING_RATE, subsamplingRate)}") | ||
| throw e | ||
| } | ||
|
|
||
|
|
||
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.
Nit: let's sort the keys.