This is an automated email from the ASF dual-hosted git repository. dongjoon pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/master by this push: new e8f87413bdd [SPARK-43782][CORE] Support log level configuration with static Spark conf e8f87413bdd is described below commit e8f87413bdd905e5e208ac8484b3b11de1623fc2 Author: jwang0306 <jared.w...@databricks.com> AuthorDate: Tue May 30 23:06:14 2023 -0700 [SPARK-43782][CORE] Support log level configuration with static Spark conf ### What changes were proposed in this pull request? This PR proposes to add a static Spark conf to configure an override to the Log4j logging level. It’s a config version of the `SparkContext.setLogLevel()` semantics, when set it’ll trigger a log level override at the beginning of `SparkContext` startup. ### Why are the changes needed? This is particularly helpful in the Spark Connect scenario where there’s no way for the client to call `setLogLevel` because `SparkContext` is not yet supported in its API, and when it connects to a platform where it can't change the Log4j config file. ### Does this PR introduce _any_ user-facing change? Yes, it adds a new public configuration `"spark.log.level"`, and by default it's unset which means there's no override. ### How was this patch tested? A new unit test in `SparkContextSuite`. Closes #41302 from jwang0306/config-logging-level. Authored-by: jwang0306 <jared.w...@databricks.com> Signed-off-by: Dongjoon Hyun <dongj...@apache.org> --- core/src/main/scala/org/apache/spark/SparkContext.scala | 8 +++++++- .../src/main/scala/org/apache/spark/internal/Logging.scala | 3 +++ .../scala/org/apache/spark/internal/config/package.scala | 14 ++++++++++++++ .../test/scala/org/apache/spark/SparkContextSuite.scala | 12 ++++++++++++ docs/configuration.md | 9 +++++++++ 5 files changed, 45 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index d119dae5c76..cf7a405f1ba 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -396,6 +396,12 @@ class SparkContext(config: SparkConf) extends Logging { try { _conf = config.clone() + _conf.get(SPARK_LOG_LEVEL).foreach { level => + if (Logging.setLogLevelPrinted) { + System.err.printf("Setting Spark log level to \"%s\".\n", level) + } + setLogLevel(level) + } _conf.validateSettings() _conf.set("spark.app.startTime", startTime.toString) @@ -2679,7 +2685,7 @@ class SparkContext(config: SparkConf) extends Logging { * various Spark features. */ object SparkContext extends Logging { - private val VALID_LOG_LEVELS = + private[spark] val VALID_LOG_LEVELS = Set("ALL", "DEBUG", "ERROR", "FATAL", "INFO", "OFF", "TRACE", "WARN") /** diff --git a/core/src/main/scala/org/apache/spark/internal/Logging.scala b/core/src/main/scala/org/apache/spark/internal/Logging.scala index 614103dee7b..b6e3914622a 100644 --- a/core/src/main/scala/org/apache/spark/internal/Logging.scala +++ b/core/src/main/scala/org/apache/spark/internal/Logging.scala @@ -139,6 +139,7 @@ trait Logging { context.setConfigLocation(url.toURI) if (!silent) { System.err.println(s"Using Spark's default log4j profile: $defaultLogProps") + Logging.setLogLevelPrinted = true } case None => System.err.println(s"Spark was unable to load $defaultLogProps") @@ -164,6 +165,7 @@ trait Logging { System.err.printf("Setting default log level to \"%s\".\n", replLevel) System.err.println("To adjust logging level use sc.setLogLevel(newLevel). " + "For SparkR, use setLogLevel(newLevel).") + Logging.setLogLevelPrinted = true } Logging.sparkShellThresholdLevel = replLevel rootLogger.getAppenders().asScala.foreach { @@ -189,6 +191,7 @@ private[spark] object Logging { @volatile private var defaultRootLevel: Level = null @volatile private var defaultSparkLog4jConfig = false @volatile private[spark] var sparkShellThresholdLevel: Level = null + @volatile private[spark] var setLogLevelPrinted: Boolean = false val initLock = new Object() try { diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index fa88606e1f6..533562af05a 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -20,6 +20,7 @@ package org.apache.spark.internal import java.util.Locale import java.util.concurrent.TimeUnit +import org.apache.spark.SparkContext import org.apache.spark.launcher.SparkLauncher import org.apache.spark.metrics.GarbageCollectionMetrics import org.apache.spark.network.shuffle.Constants @@ -1110,6 +1111,19 @@ package object config { .stringConf .createOptional + private[spark] val SPARK_LOG_LEVEL = ConfigBuilder("spark.log.level") + .doc("When set, overrides any user-defined log settings as if calling " + + "SparkContext.setLogLevel() at Spark startup. Valid log levels include: " + + SparkContext.VALID_LOG_LEVELS.mkString(",")) + .version("3.5.0") + .stringConf + .transform(_.toUpperCase(Locale.ROOT)) + .checkValue( + logLevel => SparkContext.VALID_LOG_LEVELS.contains(logLevel), + "Invalid value for 'spark.log.level'. Valid values are " + + SparkContext.VALID_LOG_LEVELS.mkString(",")) + .createOptional + private[spark] val FILES_MAX_PARTITION_BYTES = ConfigBuilder("spark.files.maxPartitionBytes") .doc("The maximum number of bytes to pack into a single partition when reading files.") .version("2.1.0") diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 7bfcbc4b0d8..135ecbee09a 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -626,6 +626,18 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu } } + test("SPARK-43782: conf to override log level") { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local") + .set(SPARK_LOG_LEVEL, "ERROR")) + assert(LogManager.getRootLogger().getLevel === Level.ERROR) + sc.stop() + + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local") + .set(SPARK_LOG_LEVEL, "TRACE")) + assert(LogManager.getRootLogger().getLevel === Level.TRACE) + sc.stop() + } + test("register and deregister Spark listener from SparkContext") { sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) val sparkListener1 = new SparkListener { } diff --git a/docs/configuration.md b/docs/configuration.md index e136fef9586..fbbb9b95f71 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -412,6 +412,15 @@ of the most common options to set are: </td> <td>2.2.0</td> </tr> +<tr> + <td><code>spark.log.level</code></td> + <td>(none)</td> + <td> + When set, overrides any user-defined log settings as if calling + <code>SparkContext.setLogLevel()</code> at Spark startup. Valid log levels include: "ALL", "DEBUG", "ERROR", "FATAL", "INFO", "OFF", "TRACE", "WARN". + </td> + <td>3.5.0</td> +</tr> <tr> <td><code>spark.driver.supervise</code></td> <td>false</td> --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org