This is an automated email from the ASF dual-hosted git repository.

viirya 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 7fd3619  [SPARK-6305][CORE][TEST][FOLLOWUP] Add LoggingSuite and some 
improvements
7fd3619 is described below

commit 7fd361973d22c4e98a008989f81cfcb2f9a41443
Author: Liang-Chi Hsieh <vii...@gmail.com>
AuthorDate: Thu Dec 23 19:41:02 2021 -0800

    [SPARK-6305][CORE][TEST][FOLLOWUP] Add LoggingSuite and some improvements
    
    ### What changes were proposed in this pull request?
    
    This patch proposes to add `LoggingSuite` back and also does some other 
improvements. In summary:
    
    1. Add `LoggingSuite` back
    2. Refactor logging related change based on community suggestion, e.g. let 
`SparkShellLoggingFilter` inherit from `AbstractFilter` instead of `Filter`.
    3. Fix maven test failures for hive-thriftserver module
    4. Fix K8S decommision integration tests which check log output
    5. A few places in code/doc which refer/mention log4j.properties
    
    ### Why are the changes needed?
    
    `LoggingSuite` was wrongly removed in previous PR. We should add it back. 
There are a few places we can also simplify the code. A few places in code 
which programmingly write out log4j properties files are also changed to log4j2 
here.
    
    ### Does this PR introduce _any_ user-facing change?
    
    No
    
    ### How was this patch tested?
    
    Pass all tests.
    
    Closes #34965 from viirya/log4j2_improvement.
    
    Authored-by: Liang-Chi Hsieh <vii...@gmail.com>
    Signed-off-by: Liang-Chi Hsieh <vii...@gmail.com>
---
 R/{log4j.properties => log4j2.properties}          |  19 +-
 R/run-tests.sh                                     |   4 +-
 .../org/apache/spark/log4j-defaults.properties     |  43 ---
 .../scala/org/apache/spark/deploy/Client.scala     |   5 +-
 .../org/apache/spark/deploy/ClientArguments.scala  |   2 +-
 .../scala/org/apache/spark/internal/Logging.scala  |  85 +----
 .../apache/spark/util/logging/DriverLogger.scala   |   1 -
 .../scala/org/apache/spark/SparkFunSuite.scala     |  25 +-
 .../org/apache/spark/internal/LoggingSuite.scala   |  64 ++++
 docs/configuration.md                              |   8 +-
 .../dev/dev-run-integration-tests.sh               |   1 -
 .../src/test/resources/log4j2.properties           |   2 +-
 .../k8s/integrationtest/DecommissionSuite.scala    | 355 ++++++++++++---------
 .../org/apache/spark/deploy/yarn/Client.scala      |  12 +-
 .../src/test/resources/log4j2.properties           |   2 +-
 .../thriftserver/HiveThriftServer2Suites.scala     |   8 +-
 .../sql/hive/thriftserver/UISeleniumSuite.scala    |  16 +-
 17 files changed, 324 insertions(+), 328 deletions(-)

diff --git a/R/log4j.properties b/R/log4j2.properties
similarity index 71%
rename from R/log4j.properties
rename to R/log4j2.properties
index cce8d91..8ed7b9f 100644
--- a/R/log4j.properties
+++ b/R/log4j2.properties
@@ -16,13 +16,16 @@
 #
 
 # Set everything to be logged to the file target/unit-tests.log
-log4j.rootCategory=INFO, file
-log4j.appender.file=org.apache.log4j.FileAppender
-log4j.appender.file.append=true
-log4j.appender.file.file=R/target/unit-tests.log
-log4j.appender.file.layout=org.apache.log4j.PatternLayout
-log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p 
%c{1}: %m%n
+rootLogger.level = info
+rootLogger.appenderRef.file.ref = File
+
+appender.file.type = File
+appender.file.name = File
+appender.file.fileName = target/unit-tests.log
+appender.file.append = true
+appender.file.layout.type = PatternLayout
+appender.file.layout.pattern = %d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n
 
 # Ignore messages below warning level from Jetty, because it's a bit verbose
-log4j.logger.org.eclipse.jetty=WARN
-org.eclipse.jetty.LEVEL=WARN
+logger.jetty.name = org.eclipse.jetty
+logger.jetty.level = warn
diff --git a/R/run-tests.sh b/R/run-tests.sh
index edc2b2b..99b7438 100755
--- a/R/run-tests.sh
+++ b/R/run-tests.sh
@@ -30,9 +30,9 @@ if [[ $(echo $SPARK_AVRO_JAR_PATH | wc -l) -eq 1 ]]; then
 fi
 
 if [ -z "$SPARK_JARS" ]; then
-  SPARK_TESTING=1 NOT_CRAN=true $FWDIR/../bin/spark-submit 
--driver-java-options "-Dlog4j.configuration=file:$FWDIR/log4j.properties" 
--conf spark.hadoop.fs.defaultFS="file:///" --conf 
spark.driver.extraJavaOptions="-Dio.netty.tryReflectionSetAccessible=true" 
--conf 
spark.executor.extraJavaOptions="-Dio.netty.tryReflectionSetAccessible=true" 
$FWDIR/pkg/tests/run-all.R 2>&1 | tee -a $LOGFILE
+  SPARK_TESTING=1 NOT_CRAN=true $FWDIR/../bin/spark-submit 
--driver-java-options "-Dlog4j.configurationFile=file:$FWDIR/log4j2.properties" 
--conf spark.hadoop.fs.defaultFS="file:///" --conf 
spark.driver.extraJavaOptions="-Dio.netty.tryReflectionSetAccessible=true" 
--conf 
spark.executor.extraJavaOptions="-Dio.netty.tryReflectionSetAccessible=true" 
$FWDIR/pkg/tests/run-all.R 2>&1 | tee -a $LOGFILE
 else
-  SPARK_TESTING=1 NOT_CRAN=true $FWDIR/../bin/spark-submit --jars $SPARK_JARS 
--driver-java-options "-Dlog4j.configuration=file:$FWDIR/log4j.properties" 
--conf spark.hadoop.fs.defaultFS="file:///" --conf 
spark.driver.extraJavaOptions="-Dio.netty.tryReflectionSetAccessible=true" 
--conf 
spark.executor.extraJavaOptions="-Dio.netty.tryReflectionSetAccessible=true" 
$FWDIR/pkg/tests/run-all.R 2>&1 | tee -a $LOGFILE
+  SPARK_TESTING=1 NOT_CRAN=true $FWDIR/../bin/spark-submit --jars $SPARK_JARS 
--driver-java-options "-Dlog4j.configurationFile=file:$FWDIR/log4j2.properties" 
--conf spark.hadoop.fs.defaultFS="file:///" --conf 
spark.driver.extraJavaOptions="-Dio.netty.tryReflectionSetAccessible=true" 
--conf 
spark.executor.extraJavaOptions="-Dio.netty.tryReflectionSetAccessible=true" 
$FWDIR/pkg/tests/run-all.R 2>&1 | tee -a $LOGFILE
 fi
 
 FAILED=$((PIPESTATUS[0]||$FAILED))
diff --git a/core/src/main/resources/org/apache/spark/log4j-defaults.properties 
b/core/src/main/resources/org/apache/spark/log4j-defaults.properties
deleted file mode 100644
index 71652d0e..0000000
--- a/core/src/main/resources/org/apache/spark/log4j-defaults.properties
+++ /dev/null
@@ -1,43 +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.
-#
-
-# Set everything to be logged to the console
-log4j.rootCategory=INFO, console
-log4j.appender.console=org.apache.log4j.ConsoleAppender
-log4j.appender.console.target=System.err
-log4j.appender.console.layout=org.apache.log4j.PatternLayout
-log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p 
%c{1}: %m%n
-
-# Set the default spark-shell log level to WARN. When running the spark-shell, 
the
-# log level for this class is used to overwrite the root logger's log level, 
so that
-# the user can have different defaults for the shell and regular Spark apps.
-log4j.logger.org.apache.spark.repl.Main=WARN
-
-# Settings to quiet third party logs that are too verbose
-log4j.logger.org.sparkproject.jetty=WARN
-log4j.logger.org.sparkproject.jetty.util.component.AbstractLifeCycle=ERROR
-log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO
-log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO
-
-# SPARK-9183: Settings to avoid annoying messages when looking up nonexistent 
UDFs
-# in SparkSQL with Hive support
-log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=FATAL
-log4j.logger.org.apache.hadoop.hive.ql.exec.FunctionRegistry=ERROR
-
-# Parquet related logging
-log4j.logger.org.apache.parquet.CorruptStatistics=ERROR
-log4j.logger.parquet.CorruptStatistics=ERROR
diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala 
b/core/src/main/scala/org/apache/spark/deploy/Client.scala
index 2f1ff54..e9482d1 100644
--- a/core/src/main/scala/org/apache/spark/deploy/Client.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala
@@ -24,7 +24,8 @@ import scala.concurrent.ExecutionContext
 import scala.reflect.ClassTag
 import scala.util.{Failure, Success}
 
-import org.apache.log4j.Logger
+import org.apache.logging.log4j.LogManager
+import org.apache.logging.log4j.core.Logger
 
 import org.apache.spark.{SecurityManager, SparkConf}
 import org.apache.spark.deploy.DeployMessages._
@@ -282,7 +283,7 @@ private[spark] class ClientApp extends SparkApplication {
     if (!conf.contains(RPC_ASK_TIMEOUT)) {
       conf.set(RPC_ASK_TIMEOUT, "10s")
     }
-    Logger.getRootLogger.setLevel(driverArgs.logLevel)
+    LogManager.getRootLogger.asInstanceOf[Logger].setLevel(driverArgs.logLevel)
 
     val rpcEnv =
       RpcEnv.create("driverClient", Utils.localHostName(), 0, conf, new 
SecurityManager(conf))
diff --git a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala 
b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala
index a410b62..af7483b 100644
--- a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala
@@ -22,7 +22,7 @@ import java.net.{URI, URISyntaxException}
 import scala.annotation.tailrec
 import scala.collection.mutable.ListBuffer
 
-import org.apache.log4j.Level
+import org.apache.logging.log4j.Level
 
 import org.apache.spark.util.{IntParam, MemoryParam, Utils}
 
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 0903019..25d0e55 100644
--- a/core/src/main/scala/org/apache/spark/internal/Logging.scala
+++ b/core/src/main/scala/org/apache/spark/internal/Logging.scala
@@ -19,10 +19,10 @@ package org.apache.spark.internal
 
 import scala.collection.JavaConverters._
 
-import org.apache.logging.log4j.{core, Level, LogManager, Marker}
+import org.apache.logging.log4j.{Level, LogManager}
 import org.apache.logging.log4j.core.{Filter, LifeCycle, LogEvent, 
LoggerContext}
 import org.apache.logging.log4j.core.appender.ConsoleAppender
-import org.apache.logging.log4j.message.Message
+import org.apache.logging.log4j.core.filter.AbstractFilter
 import org.slf4j.{Logger, LoggerFactory}
 import org.slf4j.impl.StaticLoggerBinder
 
@@ -125,12 +125,12 @@ trait Logging {
   }
 
   private def initializeLogging(isInterpreter: Boolean, silent: Boolean): Unit 
= {
-    if (!Logging.isLog4j12()) {
+    if (Logging.isLog4j2()) {
       // If Log4j is used but is not initialized, load a default properties 
file
-      val log4j12Initialized = !LogManager.getRootLogger
+      val log4j2Initialized = !LogManager.getRootLogger
         
.asInstanceOf[org.apache.logging.log4j.core.Logger].getAppenders.isEmpty
       // scalastyle:off println
-      if (!log4j12Initialized) {
+      if (!log4j2Initialized) {
         Logging.defaultSparkLog4jConfig = true
         val defaultLogProps = "org/apache/spark/log4j2-defaults.properties"
         Option(Utils.getSparkClassLoader.getResource(defaultLogProps)) match {
@@ -209,7 +209,7 @@ private[spark] object Logging {
    * initialization again.
    */
   def uninitialize(): Unit = initLock.synchronized {
-    if (!isLog4j12()) {
+    if (isLog4j2()) {
       if (defaultSparkLog4jConfig) {
         defaultSparkLog4jConfig = false
         val context = LogManager.getContext(false).asInstanceOf[LoggerContext]
@@ -224,85 +224,18 @@ private[spark] object Logging {
     this.initialized = false
   }
 
-  private def isLog4j12(): Boolean = {
+  private def isLog4j2(): Boolean = {
     // This distinguishes the log4j 1.2 binding, currently
     // org.slf4j.impl.Log4jLoggerFactory, from the log4j 2.0 binding, currently
     // org.apache.logging.slf4j.Log4jLoggerFactory
     val binderClass = StaticLoggerBinder.getSingleton.getLoggerFactoryClassStr
-    "org.slf4j.impl.Log4jLoggerFactory".equals(binderClass)
+    "org.apache.logging.slf4j.Log4jLoggerFactory".equals(binderClass)
   }
 
 
-  private class SparkShellLoggingFilter extends Filter {
+  private[spark] class SparkShellLoggingFilter extends AbstractFilter {
     private var status = LifeCycle.State.INITIALIZING
 
-    override def getOnMismatch: Filter.Result = Filter.Result.ACCEPT
-
-    override def getOnMatch: Filter.Result = Filter.Result.ACCEPT
-
-    // We don't use this with log4j2 `Marker`, currently all accept.
-    // If we need it, we should implement it.
-    override def filter(logger: core.Logger,
-        level: Level, marker: Marker, msg: String, params: Object*): 
Filter.Result =
-      Filter.Result.ACCEPT
-
-    override def filter(logger: core.Logger,
-        level: Level, marker: Marker, message: String, p0: Object): 
Filter.Result =
-      Filter.Result.ACCEPT
-
-    override def filter(logger: core.Logger,
-        level: Level, marker: Marker, message: String, p0: Object, p1: 
Object): Filter.Result =
-      Filter.Result.ACCEPT
-
-    override def filter(logger: core.Logger,
-        level: Level, marker: Marker, message: String, p0: Object, p1: Object,
-        p2: Object): Filter.Result = Filter.Result.ACCEPT
-
-    override def filter(logger: core.Logger,
-        level: Level, marker: Marker, message: String, p0: Object, p1: Object,
-        p2: Object, p3: Object): Filter.Result = Filter.Result.ACCEPT
-
-    override def filter(logger: core.Logger,
-        level: Level, marker: Marker, message: String, p0: Object, p1: Object,
-        p2: Any, p3: Any, p4: Any): Filter.Result = Filter.Result.ACCEPT
-
-    override def filter(logger: core.Logger,
-        level: Level, marker: Marker, message: String, p0: Object, p1: Object,
-        p2: Object, p3: Object, p4: Object, p5: Object): Filter.Result =
-      Filter.Result.ACCEPT
-
-    // scalastyle:off
-    override def filter(logger: core.Logger,
-        level: Level, marker: Marker, message: String, p0: Object, p1: Object,
-        p2: Object, p3: Object, p4: Object, p5: Object, p6: Object): 
Filter.Result =
-      Filter.Result.ACCEPT
-
-    override def filter(logger: core.Logger,
-        level: Level, marker: Marker, message: String, p0: Object, p1: Object,
-        p2: Object, p3: Object, p4: Object, p5: Object, p6: Object, p7: 
Object): Filter.Result =
-      Filter.Result.ACCEPT
-
-    override def filter(logger: core.Logger,
-        level: Level, marker: Marker, message: String, p0: Object, p1: Object,
-        p2: Object, p3: Object, p4: Object, p5: Object, p6: Object, p7: Object,
-        p8: Object): Filter.Result =
-      Filter.Result.ACCEPT
-
-    override def filter(logger: core.Logger,
-        level: Level, marker: Marker, message: String, p0: Object, p1: Object,
-        p2: Object, p3: Object, p4: Object, p5: Object, p6: Object, p7: Object,
-        p8: Object, p9: Object): Filter.Result =
-      Filter.Result.ACCEPT
-    // scalastyle:on
-
-    override def filter(logger: core.Logger,
-        level: Level, marker: Marker, msg: Any, t: Throwable): Filter.Result =
-      Filter.Result.ACCEPT
-
-    override def filter(logger: core.Logger,
-        level: Level, marker: Marker, msg: Message, t: Throwable): 
Filter.Result =
-      Filter.Result.ACCEPT
-
     /**
      * If sparkShellThresholdLevel is not defined, this filter is a no-op.
      * If log level of event is not equal to root level, the event is allowed. 
Otherwise,
diff --git 
a/core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala 
b/core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala
index af87d16..156f0e3 100644
--- a/core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala
+++ b/core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala
@@ -85,7 +85,6 @@ private[spark] class DriverLogger(conf: SparkConf) extends 
Logging {
       val logger = LogManager.getRootLogger().asInstanceOf[Logger]
       val fa = logger.getAppenders.get(DriverLogger.APPENDER_NAME)
       logger.removeAppender(fa)
-      fa.stop()
       Utils.tryLogNonFatalError(fa.stop())
       writer.foreach(_.closeWriter())
     } catch {
diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala 
b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala
index 66a7e8f..14a7f48 100644
--- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala
@@ -242,17 +242,13 @@ abstract class SparkFunSuite
       throw new SparkException(s"Cannot get any logger to add the appender")
     }
     val restoreLevels = loggers.map(_.getLevel)
-    loggers.foreach { logger =>
-      logger match {
-        case logger: Logger =>
-          logger.addAppender(appender)
-          appender.start()
-          if (level.isDefined) {
-            logger.setLevel(level.get)
-            logger.get().setLevel(level.get)
-          }
-        case _ =>
-          throw new SparkException(s"Cannot add appender to logger 
${logger.getName}")
+    loggers.foreach { l =>
+      val logger = l.asInstanceOf[Logger]
+      logger.addAppender(appender)
+      appender.start()
+      if (level.isDefined) {
+        logger.setLevel(level.get)
+        logger.get().setLevel(level.get)
       }
     }
     try f finally {
@@ -272,14 +268,15 @@ abstract class SparkFunSuite
     val loggingEvents = new ArrayBuffer[LogEvent]()
     private var _threshold: Level = Level.INFO
 
-    override def append(loggingEvent: LogEvent): Unit = {
-      if (loggingEvent.getLevel.isMoreSpecificThan(_threshold)) {
+    override def append(loggingEvent: LogEvent): Unit = 
loggingEvent.synchronized {
+      val copyEvent = loggingEvent.toImmutable
+      if (copyEvent.getLevel.isMoreSpecificThan(_threshold)) {
         if (loggingEvents.size >= maxEvents) {
           val loggingInfo = if (msg == "") "." else s" while logging $msg."
           throw new IllegalStateException(
             s"Number of events reached the limit of $maxEvents$loggingInfo")
         }
-        loggingEvents.append(loggingEvent.toImmutable)
+        loggingEvents.append(copyEvent)
       }
     }
 
diff --git a/core/src/test/scala/org/apache/spark/internal/LoggingSuite.scala 
b/core/src/test/scala/org/apache/spark/internal/LoggingSuite.scala
new file mode 100644
index 0000000..ef796ec
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/internal/LoggingSuite.scala
@@ -0,0 +1,64 @@
+/*
+ * 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.internal
+
+import org.apache.logging.log4j.{Level, LogManager}
+import org.apache.logging.log4j.core.{Filter, Logger}
+import org.apache.logging.log4j.core.impl.Log4jLogEvent.Builder
+import org.apache.logging.log4j.message.SimpleMessage
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.internal.Logging.SparkShellLoggingFilter
+import org.apache.spark.util.Utils
+
+class LoggingSuite extends SparkFunSuite {
+
+  test("spark-shell logging filter") {
+    val ssf = new SparkShellLoggingFilter()
+    val rootLogger = LogManager.getRootLogger().asInstanceOf[Logger]
+    val originalLevel = rootLogger.getLevel()
+    rootLogger.setLevel(Level.INFO)
+    val originalThreshold = Logging.sparkShellThresholdLevel
+    Logging.sparkShellThresholdLevel = Level.WARN
+    try {
+      val logger1 = LogManager.getLogger("a.b.c.D")
+        .asInstanceOf[Logger]
+      val logEvent1 = new Builder().setLevel(Level.INFO)
+        .setLoggerName(logger1.getName()).setMessage(new 
SimpleMessage("Test")).build()
+      // Logger's default level is not null in log4j2, and cannot be set to 
null too.
+      assert(ssf.filter(logEvent1) == Filter.Result.NEUTRAL)
+
+      // custom log level configured
+      val parentLogger = LogManager.getLogger("a.b.c")
+        .asInstanceOf[Logger]
+      parentLogger.setLevel(Level.INFO)
+      assert(ssf.filter(logEvent1) == Filter.Result.NEUTRAL)
+
+      // log level is greater than or equal to threshold level
+      val logger2 = LogManager.getLogger("a.b.E")
+        .asInstanceOf[Logger]
+      val logEvent2 = new Builder().setLevel(Level.INFO)
+        .setLoggerName(logger2.getName()).setMessage(new 
SimpleMessage("Test")).build()
+      Utils.setLogLevel(Level.INFO)
+      assert(ssf.filter(logEvent2) != Filter.Result.DENY)
+    } finally {
+      rootLogger.setLevel(originalLevel)
+      Logging.sparkShellThresholdLevel = originalThreshold
+    }
+  }
+}
diff --git a/docs/configuration.md b/docs/configuration.md
index 3d78870..2d4164f 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -28,7 +28,7 @@ Spark provides three locations to configure the system:
   system properties.
 * [Environment variables](#environment-variables) can be used to set 
per-machine settings, such as
   the IP address, through the `conf/spark-env.sh` script on each node.
-* [Logging](#configuring-logging) can be configured through `log4j.properties`.
+* [Logging](#configuring-logging) can be configured through 
`log4j2.properties`.
 
 # Spark Properties
 
@@ -422,7 +422,7 @@ of the most common options to set are:
   <td>%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n</td>
   <td>
     The layout for the driver logs that are synced to 
<code>spark.driver.log.dfsDir</code>. If this is not configured,
-    it uses the layout for the first appender defined in log4j.properties. If 
that is also not configured, driver logs
+    it uses the layout for the first appender defined in log4j2.properties. If 
that is also not configured, driver logs
     use the default layout.
   </td>
   <td>3.0.0</td>
@@ -3080,7 +3080,7 @@ Note: When running Spark on YARN in `cluster` mode, 
environment variables need t
 # Configuring Logging
 
 Spark uses [log4j](http://logging.apache.org/log4j/) for logging. You can 
configure it by adding a
-`log4j.properties` file in the `conf` directory. One way to start is to copy 
the existing
+`log4j2.properties` file in the `conf` directory. One way to start is to copy 
the existing
 `log4j2.properties.template` located there.
 
 By default, Spark adds 1 record to the MDC (Mapped Diagnostic Context): 
`mdc.taskName`, which shows something
@@ -3092,7 +3092,7 @@ The key in MDC will be the string of "mdc.$name".
 # Overriding configuration directory
 
 To specify a different configuration directory other than the default 
"SPARK_HOME/conf",
-you can set SPARK_CONF_DIR. Spark will use the configuration files 
(spark-defaults.conf, spark-env.sh, log4j.properties, etc)
+you can set SPARK_CONF_DIR. Spark will use the configuration files 
(spark-defaults.conf, spark-env.sh, log4j2.properties, etc)
 from this directory.
 
 # Inheriting Hadoop Cluster Configuration
diff --git 
a/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh
 
b/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh
index be6cf16..5f94203 100755
--- 
a/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh
+++ 
b/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh
@@ -189,7 +189,6 @@ properties+=(
   -Dspark.kubernetes.test.jvmImage=$JVM_IMAGE_NAME
   -Dspark.kubernetes.test.pythonImage=$PYTHON_IMAGE_NAME
   -Dspark.kubernetes.test.rImage=$R_IMAGE_NAME
-  -Dlog4j.logger.org.apache.spark=DEBUG
 )
 
 (
diff --git 
a/resource-managers/kubernetes/integration-tests/src/test/resources/log4j2.properties
 
b/resource-managers/kubernetes/integration-tests/src/test/resources/log4j2.properties
index 67836a4..75ac84a 100644
--- 
a/resource-managers/kubernetes/integration-tests/src/test/resources/log4j2.properties
+++ 
b/resource-managers/kubernetes/integration-tests/src/test/resources/log4j2.properties
@@ -21,7 +21,7 @@ rootLogger.appenderRef.file.ref = File
 
 appender.file.type = File
 appender.file.name = File
-appender.file.fileName = target/unit-tests.log
+appender.file.fileName = target/integration-tests.log
 appender.file.append = true
 appender.file.layout.type = PatternLayout
 appender.file.layout.pattern = %d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n
diff --git 
a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala
 
b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala
index 04834bb..9605f6c 100644
--- 
a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala
+++ 
b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala
@@ -16,8 +16,12 @@
  */
 package org.apache.spark.deploy.k8s.integrationtest
 
+import java.io.File
+import java.nio.charset.StandardCharsets
+
 import scala.collection.JavaConverters._
 
+import com.google.common.io.Files
 import io.fabric8.kubernetes.api.model.Pod
 import org.scalatest.concurrent.{Eventually, PatienceConfiguration}
 import org.scalatest.matchers.should.Matchers._
@@ -31,180 +35,215 @@ private[spark] trait DecommissionSuite { k8sSuite: 
KubernetesSuite =>
   import DecommissionSuite._
   import KubernetesSuite.k8sTestTag
 
+  def runDecommissionTest(f: () => Unit): Unit = {
+    val logConfFilePath = s"${sparkHomeDir.toFile}/conf/log4j2.properties"
+
+    try {
+      Files.write(
+        """rootLogger.level = info
+          |rootLogger.appenderRef.stdout.ref = console
+          |appender.console.type = Console
+          |appender.console.name = console
+          |appender.console.target = SYSTEM_OUT
+          |appender.console.layout.type = PatternLayout
+          |appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss} %p %c{1}: 
%m%n
+          |
+          |logger.spark.name = org.apache.spark
+          |logger.spark.level = debug
+      """.stripMargin,
+        new File(logConfFilePath),
+        StandardCharsets.UTF_8)
+
+      f()
+    } finally {
+      new File(logConfFilePath).delete()
+    }
+  }
+
   test("Test basic decommissioning", k8sTestTag) {
-    sparkAppConf
-      .set(config.DECOMMISSION_ENABLED.key, "true")
-      .set("spark.kubernetes.container.image", pyImage)
-      .set(config.STORAGE_DECOMMISSION_ENABLED.key, "true")
-      .set(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED.key, "true")
-      .set(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED.key, "true")
-      // Ensure we have somewhere to migrate our data too
-      .set("spark.executor.instances", "3")
-      // The default of 30 seconds is fine, but for testing we just want to 
get this done fast.
-      .set("spark.storage.decommission.replicationReattemptInterval", "1")
-
-    runSparkApplicationAndVerifyCompletion(
-      appResource = PYSPARK_DECOMISSIONING,
-      mainClass = "",
-      expectedDriverLogOnCompletion = Seq(
-        "Finished waiting, stopping Spark",
-        "Decommission executors",
-        "Final accumulator value is: 100"),
-      appArgs = Array.empty[String],
-      driverPodChecker = doBasicDriverPyPodCheck,
-      executorPodChecker = doBasicExecutorPyPodCheck,
-      isJVM = false,
-      pyFiles = None,
-      executorPatience = None,
-      decommissioningTest = true)
+    runDecommissionTest(() => {
+      sparkAppConf
+        .set(config.DECOMMISSION_ENABLED.key, "true")
+        .set("spark.kubernetes.container.image", pyImage)
+        .set(config.STORAGE_DECOMMISSION_ENABLED.key, "true")
+        .set(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED.key, "true")
+        .set(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED.key, "true")
+        // Ensure we have somewhere to migrate our data too
+        .set("spark.executor.instances", "3")
+        // The default of 30 seconds is fine, but for testing we just want to 
get this done fast.
+        .set("spark.storage.decommission.replicationReattemptInterval", "1")
+
+      runSparkApplicationAndVerifyCompletion(
+        appResource = PYSPARK_DECOMISSIONING,
+        mainClass = "",
+        expectedDriverLogOnCompletion = Seq(
+          "Finished waiting, stopping Spark",
+          "Decommission executors",
+          "Final accumulator value is: 100"),
+        appArgs = Array.empty[String],
+        driverPodChecker = doBasicDriverPyPodCheck,
+        executorPodChecker = doBasicExecutorPyPodCheck,
+        isJVM = false,
+        pyFiles = None,
+        executorPatience = None,
+        decommissioningTest = true)
+    })
   }
 
   test("Test basic decommissioning with shuffle cleanup", k8sTestTag) {
-    sparkAppConf
-      .set(config.DECOMMISSION_ENABLED.key, "true")
-      .set("spark.kubernetes.container.image", pyImage)
-      .set(config.STORAGE_DECOMMISSION_ENABLED.key, "true")
-      .set(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED.key, "true")
-      .set(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED.key, "true")
-      .set(config.DYN_ALLOCATION_SHUFFLE_TRACKING_ENABLED.key, "true")
-      .set(config.DYN_ALLOCATION_SHUFFLE_TRACKING_TIMEOUT.key, "400")
-      // Ensure we have somewhere to migrate our data too
-      .set("spark.executor.instances", "3")
-      // The default of 30 seconds is fine, but for testing we just want to 
get this done fast.
-      .set("spark.storage.decommission.replicationReattemptInterval", "1")
-
-    runSparkApplicationAndVerifyCompletion(
-      appResource = PYSPARK_DECOMISSIONING_CLEANUP,
-      mainClass = "",
-      expectedDriverLogOnCompletion = Seq(
-        "Finished waiting, stopping Spark",
-        "Decommission executors"),
-      appArgs = Array.empty[String],
-      driverPodChecker = doBasicDriverPyPodCheck,
-      executorPodChecker = doBasicExecutorPyPodCheck,
-      isJVM = false,
-      pyFiles = None,
-      executorPatience = None,
-      decommissioningTest = true)
+    runDecommissionTest(() => {
+      sparkAppConf
+        .set(config.DECOMMISSION_ENABLED.key, "true")
+        .set("spark.kubernetes.container.image", pyImage)
+        .set(config.STORAGE_DECOMMISSION_ENABLED.key, "true")
+        .set(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED.key, "true")
+        .set(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED.key, "true")
+        .set(config.DYN_ALLOCATION_SHUFFLE_TRACKING_ENABLED.key, "true")
+        .set(config.DYN_ALLOCATION_SHUFFLE_TRACKING_TIMEOUT.key, "400")
+        // Ensure we have somewhere to migrate our data too
+        .set("spark.executor.instances", "3")
+        // The default of 30 seconds is fine, but for testing we just want to 
get this done fast.
+        .set("spark.storage.decommission.replicationReattemptInterval", "1")
+
+      runSparkApplicationAndVerifyCompletion(
+        appResource = PYSPARK_DECOMISSIONING_CLEANUP,
+        mainClass = "",
+        expectedDriverLogOnCompletion = Seq(
+          "Finished waiting, stopping Spark",
+          "Decommission executors"),
+        appArgs = Array.empty[String],
+        driverPodChecker = doBasicDriverPyPodCheck,
+        executorPodChecker = doBasicExecutorPyPodCheck,
+        isJVM = false,
+        pyFiles = None,
+        executorPatience = None,
+        decommissioningTest = true)
+    })
   }
 
   test("Test decommissioning with dynamic allocation & shuffle cleanups", 
k8sTestTag) {
-    sparkAppConf
-      .set(config.DECOMMISSION_ENABLED.key, "true")
-      .set("spark.kubernetes.container.image", pyImage)
-      .set(config.STORAGE_DECOMMISSION_ENABLED.key, "true")
-      .set(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED.key, "true")
-      .set(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED.key, "true")
-      .set(config.DYN_ALLOCATION_SHUFFLE_TRACKING_ENABLED.key, "true")
-      .set(config.DYN_ALLOCATION_SHUFFLE_TRACKING_TIMEOUT.key, "30")
-      .set(config.DYN_ALLOCATION_CACHED_EXECUTOR_IDLE_TIMEOUT.key, "30")
-      .set(config.DYN_ALLOCATION_EXECUTOR_IDLE_TIMEOUT.key, "5")
-      .set(config.DYN_ALLOCATION_MIN_EXECUTORS.key, "1")
-      .set(config.DYN_ALLOCATION_INITIAL_EXECUTORS.key, "2")
-      .set(config.DYN_ALLOCATION_ENABLED.key, "true")
-      // The default of 30 seconds is fine, but for testing we just want to
-      // give enough time to validate the labels are set.
-      .set("spark.storage.decommission.replicationReattemptInterval", "75")
-      // Configure labels for decommissioning pods.
-      .set("spark.kubernetes.executor.decommmissionLabel", "solong")
-      .set("spark.kubernetes.executor.decommmissionLabelValue", "cruelworld")
-
-    // This is called on all exec pods but we only care about exec 0 since 
it's the "first."
-    // We only do this inside of this test since the other tests trigger k8s 
side deletes where we
-    // do not apply labels.
-    def checkFirstExecutorPodGetsLabeled(pod: Pod): Unit = {
-      if (pod.getMetadata.getName.endsWith("-1")) {
-        val client = kubernetesTestComponents.kubernetesClient
-        // The label will be added eventually, but k8s objects don't refresh.
-        Eventually.eventually(
-          PatienceConfiguration.Timeout(Span(1200, Seconds)),
-          PatienceConfiguration.Interval(Span(1, Seconds))) {
-
-          val currentPod = client.pods().withName(pod.getMetadata.getName).get
-          val labels = currentPod.getMetadata.getLabels.asScala
-
-          labels should not be (null)
-          labels should (contain key ("solong") and contain value 
("cruelworld"))
+    runDecommissionTest(() => {
+      sparkAppConf
+        .set(config.DECOMMISSION_ENABLED.key, "true")
+        .set("spark.kubernetes.container.image", pyImage)
+        .set(config.STORAGE_DECOMMISSION_ENABLED.key, "true")
+        .set(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED.key, "true")
+        .set(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED.key, "true")
+        .set(config.DYN_ALLOCATION_SHUFFLE_TRACKING_ENABLED.key, "true")
+        .set(config.DYN_ALLOCATION_SHUFFLE_TRACKING_TIMEOUT.key, "30")
+        .set(config.DYN_ALLOCATION_CACHED_EXECUTOR_IDLE_TIMEOUT.key, "30")
+        .set(config.DYN_ALLOCATION_EXECUTOR_IDLE_TIMEOUT.key, "5")
+        .set(config.DYN_ALLOCATION_MIN_EXECUTORS.key, "1")
+        .set(config.DYN_ALLOCATION_INITIAL_EXECUTORS.key, "2")
+        .set(config.DYN_ALLOCATION_ENABLED.key, "true")
+        // The default of 30 seconds is fine, but for testing we just want to
+        // give enough time to validate the labels are set.
+        .set("spark.storage.decommission.replicationReattemptInterval", "75")
+        // Configure labels for decommissioning pods.
+        .set("spark.kubernetes.executor.decommmissionLabel", "solong")
+        .set("spark.kubernetes.executor.decommmissionLabelValue", "cruelworld")
+
+      // This is called on all exec pods but we only care about exec 0 since 
it's the "first."
+      // We only do this inside of this test since the other tests trigger k8s 
side deletes where we
+      // do not apply labels.
+      def checkFirstExecutorPodGetsLabeled(pod: Pod): Unit = {
+        if (pod.getMetadata.getName.endsWith("-1")) {
+          val client = kubernetesTestComponents.kubernetesClient
+          // The label will be added eventually, but k8s objects don't refresh.
+          Eventually.eventually(
+            PatienceConfiguration.Timeout(Span(1200, Seconds)),
+            PatienceConfiguration.Interval(Span(1, Seconds))) {
+
+            val currentPod = 
client.pods().withName(pod.getMetadata.getName).get
+            val labels = currentPod.getMetadata.getLabels.asScala
+
+            labels should not be (null)
+            labels should (contain key ("solong") and contain value 
("cruelworld"))
+          }
         }
+        doBasicExecutorPyPodCheck(pod)
       }
-      doBasicExecutorPyPodCheck(pod)
-    }
 
-    runSparkApplicationAndVerifyCompletion(
-      appResource = PYSPARK_SCALE,
-      mainClass = "",
-      expectedDriverLogOnCompletion = Seq(
-        "Finished waiting, stopping Spark",
-        "Decommission executors",
-        "Remove reason statistics: (gracefully decommissioned: 1, decommision 
unfinished: 0, " +
-          "driver killed: 0, unexpectedly exited: 0)."),
-      appArgs = Array.empty[String],
-      driverPodChecker = doBasicDriverPyPodCheck,
-      executorPodChecker = checkFirstExecutorPodGetsLabeled,
-      isJVM = false,
-      pyFiles = None,
-      executorPatience = Some(None, Some(DECOMMISSIONING_FINISHED_TIMEOUT)),
-      decommissioningTest = false)
+      runSparkApplicationAndVerifyCompletion(
+        appResource = PYSPARK_SCALE,
+        mainClass = "",
+        expectedDriverLogOnCompletion = Seq(
+          "Finished waiting, stopping Spark",
+          "Decommission executors",
+          "Remove reason statistics: (gracefully decommissioned: 1, 
decommision unfinished: 0, " +
+            "driver killed: 0, unexpectedly exited: 0)."),
+        appArgs = Array.empty[String],
+        driverPodChecker = doBasicDriverPyPodCheck,
+        executorPodChecker = checkFirstExecutorPodGetsLabeled,
+        isJVM = false,
+        pyFiles = None,
+        executorPatience = Some(None, Some(DECOMMISSIONING_FINISHED_TIMEOUT)),
+        decommissioningTest = false)
+    })
   }
 
   test("Test decommissioning timeouts", k8sTestTag) {
-    sparkAppConf
-      .set(config.DECOMMISSION_ENABLED.key, "true")
-      .set("spark.kubernetes.container.image", pyImage)
-      .set(config.STORAGE_DECOMMISSION_ENABLED.key, "true")
-      .set(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED.key, "true")
-      .set(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED.key, "true")
-      // Ensure we have somewhere to migrate our data too
-      .set("spark.executor.instances", "3")
-      // Set super high so the timeout is triggered
-      .set("spark.storage.decommission.replicationReattemptInterval", 
"8640000")
-      // Set super low so the timeout is triggered
-      .set(config.EXECUTOR_DECOMMISSION_FORCE_KILL_TIMEOUT.key, "10")
-
-    runSparkApplicationAndVerifyCompletion(
-      appResource = PYSPARK_DECOMISSIONING,
-      mainClass = "",
-      expectedDriverLogOnCompletion = Seq(
-        "Finished waiting, stopping Spark",
-        "Decommission executors",
-        "failed to decommission in 10, killing",
-        "killed by driver."),
-      appArgs = Array.empty[String],
-      driverPodChecker = doBasicDriverPyPodCheck,
-      executorPodChecker = doBasicExecutorPyPodCheck,
-      isJVM = false,
-      pyFiles = None,
-      executorPatience = None,
-      decommissioningTest = true)
+    runDecommissionTest(() => {
+      sparkAppConf
+        .set(config.DECOMMISSION_ENABLED.key, "true")
+        .set("spark.kubernetes.container.image", pyImage)
+        .set(config.STORAGE_DECOMMISSION_ENABLED.key, "true")
+        .set(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED.key, "true")
+        .set(config.STORAGE_DECOMMISSION_RDD_BLOCKS_ENABLED.key, "true")
+        // Ensure we have somewhere to migrate our data too
+        .set("spark.executor.instances", "3")
+        // Set super high so the timeout is triggered
+        .set("spark.storage.decommission.replicationReattemptInterval", 
"8640000")
+        // Set super low so the timeout is triggered
+        .set(config.EXECUTOR_DECOMMISSION_FORCE_KILL_TIMEOUT.key, "10")
+
+      runSparkApplicationAndVerifyCompletion(
+        appResource = PYSPARK_DECOMISSIONING,
+        mainClass = "",
+        expectedDriverLogOnCompletion = Seq(
+          "Finished waiting, stopping Spark",
+          "Decommission executors",
+          "failed to decommission in 10, killing",
+          "killed by driver."),
+        appArgs = Array.empty[String],
+        driverPodChecker = doBasicDriverPyPodCheck,
+        executorPodChecker = doBasicExecutorPyPodCheck,
+        isJVM = false,
+        pyFiles = None,
+        executorPatience = None,
+        decommissioningTest = true)
+    })
   }
 
   test("SPARK-37576: Rolling decommissioning", k8sTestTag) {
-    sparkAppConf
-      .set("spark.kubernetes.container.image", pyImage)
-      .set(config.DECOMMISSION_ENABLED.key, "true")
-      .set(PLUGINS.key, 
"org.apache.spark.scheduler.cluster.k8s.ExecutorRollPlugin")
-      .set("spark.kubernetes.executor.rollInterval", "30s")
-      .set("spark.kubernetes.executor.rollPolicy", "ID")
-
-    runSparkApplicationAndVerifyCompletion(
-      appResource = PythonTestsSuite.PYSPARK_PI,
-      mainClass = "",
-      expectedDriverLogOnCompletion = Seq(
-        "Initialized driver component for plugin " +
-          "org.apache.spark.scheduler.cluster.k8s.ExecutorRollPlugin",
-        "Ask to decommission executor 1",
-        "Removed 1 successfully in removeExecutor",
-        "Going to request 1 executors",
-        "Ask to decommission executor 2",
-        "Removed 2 successfully in removeExecutor",
-        "Going to request 1 executors"),
-      appArgs = Array("10000"),
-      driverPodChecker = doBasicDriverPyPodCheck,
-      executorPodChecker = doBasicExecutorPyPodCheck,
-      isJVM = false,
-      pyFiles = None,
-      executorPatience = None,
-      decommissioningTest = true)
+    runDecommissionTest(() => {
+      sparkAppConf
+        .set("spark.kubernetes.container.image", pyImage)
+        .set(config.DECOMMISSION_ENABLED.key, "true")
+        .set(PLUGINS.key, 
"org.apache.spark.scheduler.cluster.k8s.ExecutorRollPlugin")
+        .set("spark.kubernetes.executor.rollInterval", "30s")
+        .set("spark.kubernetes.executor.rollPolicy", "ID")
+
+      runSparkApplicationAndVerifyCompletion(
+        appResource = PythonTestsSuite.PYSPARK_PI,
+        mainClass = "",
+        expectedDriverLogOnCompletion = Seq(
+          "Initialized driver component for plugin " +
+            "org.apache.spark.scheduler.cluster.k8s.ExecutorRollPlugin",
+          "Ask to decommission executor 1",
+          "Removed 1 successfully in removeExecutor",
+          "Going to request 1 executors",
+          "Ask to decommission executor 2",
+          "Removed 2 successfully in removeExecutor",
+          "Going to request 1 executors"),
+        appArgs = Array("10000"),
+        driverPodChecker = doBasicDriverPyPodCheck,
+        executorPodChecker = doBasicExecutorPyPodCheck,
+        isJVM = false,
+        pyFiles = None,
+        executorPatience = None,
+        decommissioningTest = true)
+    })
   }
 }
 
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 e2abded..ca4fbbb 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
@@ -839,16 +839,18 @@ private[spark] class Client(
     try {
       confStream.setLevel(0)
 
-      // Upload $SPARK_CONF_DIR/log4j.properties file to the distributed cache 
to make sure that
+      // Upload $SPARK_CONF_DIR/log4j2 configuration file to the distributed 
cache to make sure that
       // the executors will use the latest configurations instead of the 
default values. This is
-      // required when user changes log4j.properties directly to set the log 
configurations. If
+      // required when user changes log4j2 configuration directly to set the 
log configurations. If
       // configuration file is provided through --files then executors will be 
taking configurations
-      // from --files instead of $SPARK_CONF_DIR/log4j.properties.
+      // from --files instead of $SPARK_CONF_DIR/log4j2 configuration file.
 
       // Also upload metrics.properties to distributed cache if exists in 
classpath.
       // If user specify this file using --files then executors will use the 
one
       // from --files instead.
-      for { prop <- Seq("log4j.properties", "metrics.properties")
+      val log4j2ConfigFiles = Seq("log4j2.yaml", "log4j2.yml", "log4j2.json", 
"log4j2.jsn",
+        "log4j2.xml", "log4j2.properties")
+      for { prop <- log4j2ConfigFiles ++ Seq("metrics.properties")
             url <- Option(Utils.getContextOrSparkClassLoader.getResource(prop))
             if url.getProtocol == "file" } {
         val file = new File(url.getPath())
@@ -1047,7 +1049,7 @@ private[spark] class Client(
       }
     }
 
-    // For log4j configuration to reference
+    // For log4j2 configuration to reference
     javaOpts += ("-Dspark.yarn.app.container.log.dir=" + 
ApplicationConstants.LOG_DIR_EXPANSION_VAR)
 
     val userClass =
diff --git a/sql/hive-thriftserver/src/test/resources/log4j2.properties 
b/sql/hive-thriftserver/src/test/resources/log4j2.properties
index 1a651fa..58e18af 100644
--- a/sql/hive-thriftserver/src/test/resources/log4j2.properties
+++ b/sql/hive-thriftserver/src/test/resources/log4j2.properties
@@ -30,7 +30,7 @@ appender.console.layout.pattern = %d{HH:mm:ss.SSS} %p %c: %m%n
 appender.console.filter.1.type = Filters
 
 appender.console.filter.1.a.type = ThresholdFilter
-appender.console.filter.1.a.level = info
+appender.console.filter.1.a.level = warn
 
 # SPARK-34128: Suppress undesirable TTransportException warnings, due to 
THRIFT-4805
 appender.console.filter.1.b.type = MarkerFilter
diff --git 
a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
 
b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
index 6f1a439..bcb8ef0 100644
--- 
a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
+++ 
b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
@@ -1215,20 +1215,20 @@ abstract class HiveThriftServer2TestBase extends 
SparkFunSuite with BeforeAndAft
     }
 
     val driverClassPath = {
-      // Writes a temporary log4j.properties and prepend it to driver 
classpath, so that it
+      // Writes a temporary log4j2.properties and prepend it to driver 
classpath, so that it
       // overrides all other potential log4j configurations contained in other 
dependency jar files.
       val tempLog4jConf = Utils.createTempDir().getCanonicalPath
 
       Files.write(
         """rootLogger.level = info
-          |rootLogger.appenderRef.file.ref = console
+          |rootLogger.appenderRef.stdout.ref = console
           |appender.console.type = Console
           |appender.console.name = console
           |appender.console.target = SYSTEM_ERR
           |appender.console.layout.type = PatternLayout
           |appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss} %p %c{1}: 
%m%n
         """.stripMargin,
-        new File(s"$tempLog4jConf/log4j.properties"),
+        new File(s"$tempLog4jConf/log4j2.properties"),
         StandardCharsets.UTF_8)
 
       tempLog4jConf
@@ -1244,7 +1244,7 @@ abstract class HiveThriftServer2TestBase extends 
SparkFunSuite with BeforeAndAft
        |  --hiveconf ${ConfVars.LOCALSCRATCHDIR}=$lScratchDir
        |  --hiveconf $portConf=0
        |  --driver-class-path $driverClassPath
-       |  --driver-java-options -Dlog4j.debug
+       |  --driver-java-options -Dlog4j2.debug
        |  --conf spark.ui.enabled=false
        |  ${extraConf.mkString("\n")}
      """.stripMargin.split("\\s+").toSeq
diff --git 
a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala
 
b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala
index 0a31bc9..5d94be3 100644
--- 
a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala
+++ 
b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala
@@ -69,18 +69,20 @@ class UISeleniumSuite
     }
 
     val driverClassPath = {
-      // Writes a temporary log4j.properties and prepend it to driver 
classpath, so that it
+      // Writes a temporary log4j2.properties and prepend it to driver 
classpath, so that it
       // overrides all other potential log4j configurations contained in other 
dependency jar files.
       val tempLog4jConf = 
org.apache.spark.util.Utils.createTempDir().getCanonicalPath
 
       Files.write(
-        """log4j.rootCategory=INFO, console
-          |log4j.appender.console=org.apache.log4j.ConsoleAppender
-          |log4j.appender.console.target=System.err
-          |log4j.appender.console.layout=org.apache.log4j.PatternLayout
-          |log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd 
HH:mm:ss} %p %c{1}: %m%n
+        """rootLogger.level = info
+          |rootLogger.appenderRef.file.ref = console
+          |appender.console.type = Console
+          |appender.console.name = console
+          |appender.console.target = SYSTEM_ERR
+          |appender.console.layout.type = PatternLayout
+          |appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss} %p %c{1}: 
%m%n
         """.stripMargin,
-        new File(s"$tempLog4jConf/log4j.properties"),
+        new File(s"$tempLog4jConf/log4j2.properties"),
         StandardCharsets.UTF_8)
 
       tempLog4jConf

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to