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 156f0e393bfbe..4854a84eb5680 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 @@ -61,12 +61,23 @@ private[spark] class DriverLogger(conf: SparkConf) extends Logging { PatternLayout.newBuilder().withPattern(DEFAULT_LAYOUT).build() } val config = logger.getContext.getConfiguration() - val fa = Log4jFileAppender.createAppender(localLogFile, "false", "false", - DriverLogger.APPENDER_NAME, "true", "false", "false", "4000", layout, null, - "false", null, config); + def log4jFileAppender() = { + // SPARK-37853: We can't use the chained API invocation mode because + // `AbstractFilterable.Builder.asBuilder()` method will return `Any` in Scala. + val builder: Log4jFileAppender.Builder[_] = Log4jFileAppender.newBuilder() + builder.withAppend(false) + builder.withBufferedIo(false) + builder.setConfiguration(config) + builder.withFileName(localLogFile) + builder.setIgnoreExceptions(false) + builder.setLayout(layout) + builder.setName(DriverLogger.APPENDER_NAME) + builder.build() + } + val fa = log4jFileAppender() logger.addAppender(fa) fa.start() - logInfo(s"Added a local log appender at: ${localLogFile}") + logInfo(s"Added a local log appender at: $localLogFile") } def startSync(hadoopConf: Configuration): Unit = { diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala index 273ffeb547de5..81b40a324d0de 100644 --- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark -// scalastyle:off import java.io.File import java.nio.file.Path import java.util.{Locale, TimeZone} @@ -27,8 +26,9 @@ import scala.collection.mutable.ArrayBuffer import org.apache.commons.io.FileUtils import org.apache.logging.log4j._ -import org.apache.logging.log4j.core.appender.AbstractAppender import org.apache.logging.log4j.core.{LogEvent, Logger, LoggerContext} +import org.apache.logging.log4j.core.appender.AbstractAppender +import org.apache.logging.log4j.core.config.Property import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, BeforeAndAfterEach, Failed, Outcome} import org.scalatest.funsuite.AnyFunSuite @@ -265,7 +265,7 @@ abstract class SparkFunSuite } class LogAppender(msg: String = "", maxEvents: Int = 1000) - extends AbstractAppender("logAppender", null, null) { + extends AbstractAppender("logAppender", null, null, true, Property.EMPTY_ARRAY) { private val _loggingEvents = new ArrayBuffer[LogEvent]() private var _threshold: Level = Level.INFO