Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
28 changes: 16 additions & 12 deletions core/src/main/scala/org/apache/spark/internal/Logging.scala
Original file line number Diff line number Diff line change
Expand Up @@ -116,6 +116,11 @@ trait Logging {
false
}

// For testing
def initializeForcefully(isInterpreter: Boolean, silent: Boolean): Unit = {
initializeLogging(isInterpreter, silent)
}

private def initializeLogging(isInterpreter: Boolean, silent: Boolean): Unit = {
// Don't use a logger in here, as this is itself occurring during initialization of a logger
// If Log4j 1.2 is being used, but is not initialized, load a default properties file
Expand Down Expand Up @@ -230,19 +235,18 @@ private class SparkShellLoggingFilter extends Filter {
*/
def decide(loggingEvent: LoggingEvent): Int = {
if (Logging.sparkShellThresholdLevel == null) {
return Filter.NEUTRAL
}
val rootLevel = LogManager.getRootLogger().getLevel()
if (!loggingEvent.getLevel().eq(rootLevel)) {
return Filter.NEUTRAL
}
var logger = loggingEvent.getLogger()
while (logger.getParent() != null) {
if (logger.getLevel() != null) {
return Filter.NEUTRAL
Filter.NEUTRAL
} else if (loggingEvent.getLevel.isGreaterOrEqual(Logging.sparkShellThresholdLevel)) {
Filter.NEUTRAL
} else {
var logger = loggingEvent.getLogger()
while (logger.getParent() != null) {
if (logger.getLevel != null || logger.getAllAppenders.hasMoreElements) {
return Filter.NEUTRAL
}
logger = logger.getParent()
}
logger = logger.getParent()
Filter.DENY
}
return Filter.DENY
}
}
12 changes: 4 additions & 8 deletions core/src/test/scala/org/apache/spark/internal/LoggingSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -33,18 +33,14 @@ class LoggingSuite extends SparkFunSuite {
val originalThreshold = Logging.sparkShellThresholdLevel
Logging.sparkShellThresholdLevel = Level.WARN
try {
val logger = Logger.getLogger("a.b.c.D")
val logEvent = new LoggingEvent(logger.getName(), logger, Level.INFO, "Test", null)
assert(ssf.decide(logEvent) === Filter.DENY)

// log level is less than threshold level but different from root level
val logEvent1 = new LoggingEvent(logger.getName(), logger, Level.DEBUG, "Test", null)
assert(ssf.decide(logEvent1) != Filter.DENY)
val logger1 = Logger.getLogger("a.b.c.D")
val logEvent1 = new LoggingEvent(logger1.getName(), logger1, Level.INFO, "Test", null)
assert(ssf.decide(logEvent1) == Filter.DENY)

// custom log level configured
val parentLogger = Logger.getLogger("a.b.c")
parentLogger.setLevel(Level.INFO)
assert(ssf.decide(logEvent) != Filter.DENY)
assert(ssf.decide(logEvent1) != Filter.DENY)

// log level is greater than or equal to threshold level
val logger2 = Logger.getLogger("a.b.E")
Expand Down
114 changes: 113 additions & 1 deletion repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -18,13 +18,15 @@
package org.apache.spark.repl

import java.io._
import java.nio.file.Files

import scala.tools.nsc.interpreter.SimpleReader

import org.apache.log4j.{Level, LogManager}
import org.apache.log4j.{Level, LogManager, PropertyConfigurator}
import org.scalatest.BeforeAndAfterAll

import org.apache.spark.{SparkContext, SparkFunSuite}
import org.apache.spark.internal.Logging
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION

Expand Down Expand Up @@ -297,4 +299,114 @@ class ReplSuite extends SparkFunSuite with BeforeAndAfterAll {
assertContains("successful", output)
}

test("SPARK-30167: Log4j configuration for REPL should override root logger properly") {
val testConfiguration =
"""
|# 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 log level for this class to WARN same as the default setting.
|log4j.logger.org.apache.spark.repl.Main=ERROR
|""".stripMargin

val log4jprops = Files.createTempFile("log4j.properties.d", "log4j.properties")
Files.write(log4jprops, testConfiguration.getBytes)

val originalRootLogger = LogManager.getRootLogger
val originalRootAppender = originalRootLogger.getAppender("file")
val originalStderr = System.err
val originalReplThresholdLevel = Logging.sparkShellThresholdLevel

val replLoggerLogMessage = "Log level for REPL: "
val warnLogMessage1 = "warnLogMessage1 should not be output"
val errorLogMessage1 = "errorLogMessage1 should be output"
val infoLogMessage1 = "infoLogMessage2 should be output"
val infoLogMessage2 = "infoLogMessage3 should be output"

val out = try {
PropertyConfigurator.configure(log4jprops.toAbsolutePath.toString)

// Re-initialization is needed to set SparkShellLoggingFilter to ConsoleAppender
Main.initializeForcefully(true, false)
runInterpreter("local",
s"""
|import java.io.{ByteArrayOutputStream, PrintStream}
|
|import org.apache.log4j.{ConsoleAppender, Level, LogManager}
|
|val replLogger = LogManager.getLogger("${Main.getClass.getName.stripSuffix("$")}")
|
|// Log level for REPL is expected to be ERROR
|"$replLoggerLogMessage" + replLogger.getLevel()
|
|val bout = new ByteArrayOutputStream()
|
|// Configure stderr to let log messages output to ByteArrayOutputStream.
|val defaultErrStream: PrintStream = System.err
|try {
| System.setErr(new PrintStream(bout))
|
| // Reconfigure ConsoleAppender to reflect the stderr setting.
| val consoleAppender =
| LogManager.getRootLogger.getAllAppenders.nextElement.asInstanceOf[ConsoleAppender]
| consoleAppender.activateOptions()
|
| // customLogger1 is not explicitly configured neither its log level nor appender
| // so this inherits the settings of rootLogger
| // but ConsoleAppender can use a different log level.
| val customLogger1 = LogManager.getLogger("customLogger1")
| customLogger1.warn("$warnLogMessage1")
| customLogger1.error("$errorLogMessage1")
|
| // customLogger2 is explicitly configured its log level as INFO
| // so info level messages logged via customLogger2 should be output.
| val customLogger2 = LogManager.getLogger("customLogger2")
| customLogger2.setLevel(Level.INFO)
| customLogger2.info("$infoLogMessage1")
|
| // customLogger2 is explicitly configured its log level
| // so its child should inherit the settings.
| val customLogger3 = LogManager.getLogger("customLogger2.child")
| customLogger3.info("$infoLogMessage2")
|
| // echo log messages
| bout.toString
|} finally {
| System.setErr(defaultErrStream)
|}
|""".stripMargin)
} finally {
// Restore log4j settings for this suite
val log4jproperties = Thread.currentThread()
.getContextClassLoader.getResource("log4j.properties")
LogManager.resetConfiguration()
PropertyConfigurator.configure(log4jproperties)
Logging.sparkShellThresholdLevel = originalReplThresholdLevel
}

// Ensure stderr configuration is successfully restored.
assert(originalStderr eq System.err)

// Ensure log4j settings are successfully restored.
val restoredRootLogger = LogManager.getRootLogger
val restoredRootAppender = restoredRootLogger.getAppender("file")
assert(originalRootAppender.getClass == restoredRootAppender.getClass)
assert(originalRootLogger.getLevel == restoredRootLogger.getLevel)

// Ensure loggers added in this test case are successfully removed.
assert(LogManager.getLogger("customLogger2").getLevel == null)
assert(LogManager.getLogger("customLogger2.child").getLevel == null)

// Ensure log level threshold for REPL is ERROR.
assertContains(replLoggerLogMessage + "ERROR", out)

assertDoesNotContain(warnLogMessage1, out)
assertContains(errorLogMessage1, out)
assertContains(infoLogMessage1, out)
assertContains(infoLogMessage2, out)
}
}