You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "grundprinzip (via GitHub)" <gi...@apache.org> on 2023/07/02 18:48:07 UTC

[GitHub] [spark] grundprinzip commented on a diff in pull request #41746: [SPARK-44198][CORE] Fix inconsistent Log Level Setting between Spark Driver and Executors

grundprinzip commented on code in PR #41746:
URL: https://github.com/apache/spark/pull/41746#discussion_r1249693116


##########
core/src/main/scala/org/apache/spark/util/Utils.scala:
##########
@@ -2464,16 +2465,27 @@ private[spark] object Utils extends Logging with SparkClassUtils {
    * configure a new log4j level
    */
   def setLogLevel(l: Level): Unit = {
-    val ctx = LogManager.getContext(false).asInstanceOf[LoggerContext]
-    val config = ctx.getConfiguration()
-    val loggerConfig = config.getLoggerConfig(LogManager.ROOT_LOGGER_NAME)
+    val (ctx, loggerConfig) = getLogContext
     loggerConfig.setLevel(l)
     ctx.updateLoggers()
 
     // Setting threshold to null as rootLevel will define log level for spark-shell
     Logging.sparkShellThresholdLevel = null
   }
 
+  private lazy val getLogContext: (LoggerContext, LoggerConfig) = {
+    val ctx = LogManager.getContext(false).asInstanceOf[LoggerContext]

Review Comment:
   Is this null safe?



##########
core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala:
##########
@@ -311,13 +311,24 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
               decommissionExecutors(Array((executorId, v._1)), v._2, v._3)
               unknownExecutorsPendingDecommission.invalidate(executorId)
             })
+          // propagate current log level to new executor only if flag is true
+          if (conf.get(EXECUTOR_ALLOW_SYNC_LOG_LEVEL)) {
+            data.executorEndpoint.send(RefreshExecutor(Map("logLevel" -> Utils.getLogLevel)))
+          }

Review Comment:
   Conceptually, I agree with @mridulm. The only thing I'm wondering is if it creates some undesired side-effects if the user turns on the "sync" flag and then sets the log level to overwrite the executor configuration.
   
   I don't think this is a real case though.



##########
core/src/main/scala/org/apache/spark/SparkContext.scala:
##########
@@ -384,20 +384,22 @@ class SparkContext(config: SparkConf) extends Logging {
   def setLogLevel(logLevel: String): Unit = {
     // let's allow lowercase or mixed case too
     val upperCased = logLevel.toUpperCase(Locale.ROOT)
-    require(SparkContext.VALID_LOG_LEVELS.contains(upperCased),
+    require(
+      SparkContext.VALID_LOG_LEVELS.contains(upperCased),
       s"Supplied level $logLevel did not match one of:" +
         s" ${SparkContext.VALID_LOG_LEVELS.mkString(",")}")
-    Utils.setLogLevel(Level.toLevel(upperCased))
+    // Update only if new log level is not same as current log level
+    if (upperCased != Utils.getLogLevel) {
+      Utils.setLogLevel(Level.toLevel(upperCased))
+    // Inform all executors about the change

Review Comment:
   nit: indent



##########
core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala:
##########
@@ -49,6 +49,10 @@ private[spark] object CoarseGrainedClusterMessages {
   case class KillExecutorsOnHost(host: String)
     extends CoarseGrainedClusterMessage
 
+  case class RefreshExecutors(msg: Map[String, String]) extends CoarseGrainedClusterMessage
+
+  case class RefreshExecutor(msg: Map[String, String]) extends CoarseGrainedClusterMessage

Review Comment:
   +1 that the messasge name is too generic. It should reflect the intent of the caller. I would prefer `Update*` as well



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org