Skip to content

Commit

Permalink
[SPARK-30167][REPL] Log4j configuration for REPL can't override the r…
Browse files Browse the repository at this point in the history
…oot logger properly

### What changes were proposed in this pull request?

In the current implementation of `SparkShellLoggingFilter`, if the log level of the root logger and the log level of a message are different, whether a message should logged is decided based on log4j's configuration but whether the message should be output to the REPL's console is not cared.
So, if the log level of the root logger is `DEBUG`, the log level of REPL's logger is `WARN` and the log level of a message is `INFO`, the message will output to the REPL's console even though `INFO < WARN`.
https://github.com/apache/spark/pull/26798/files#diff-bfd5810d8aa78ad90150e806d830bb78L237

The ideal behavior should be like as follows and implemented them in this change.

1. If the log level of a message is greater than or equal to the log level of the root logger, the message should be logged but whether the message is output to the REPL's console should be decided based on whether the log level of the message is greater than or equal to the log level of the REPL's logger.

2. If a log level or custom appenders are explicitly defined for a category, whether a log message via the logger corresponding to the category is logged and output to the REPL's console should be decided baed on the log level of the category.
We can confirm whether a log level or appenders are explicitly set to a logger for a category by `Logger#getLevel` and `Logger#getAllAppenders.hasMoreElements`.

### Why are the changes needed?

This is a bug breaking a compatibility.

apache#9816 enabled REPL's log4j configuration to override root logger but apache#23675 seems to have broken the feature.
You can see one example when you modifies the default log4j configuration like as follows.
```
# Change the log level for rootCategory to DEBUG
log4j.rootCategory=DEBUG, console

...
# The log level for repl.Main remains WARN
log4j.logger.org.apache.spark.repl.Main=WARN
```
If you launch REPL with the configuration, INFO level logs appear even though the log level for REPL is WARN.
```
・・・

19/12/08 23:31:38 INFO Utils: Successfully started service 'sparkDriver' on port 33083.
19/12/08 23:31:38 INFO SparkEnv: Registering MapOutputTracker
19/12/08 23:31:38 INFO SparkEnv: Registering BlockManagerMaster
19/12/08 23:31:38 INFO BlockManagerMasterEndpoint: Using org.apache.spark.storage.DefaultTopologyMapper for getting topology information
19/12/08 23:31:38 INFO BlockManagerMasterEndpoint: BlockManagerMasterEndpoint up
19/12/08 23:31:38 INFO SparkEnv: Registering BlockManagerMasterHeartbeat

・・・
```
Before apache#23675 was applied, those INFO level logs are not shown with the same log4j.properties.

### Does this PR introduce any user-facing change?

Yes. The logging behavior for REPL is fixed.

### How was this patch tested?

Manual test and newly added unit test.

Closes apache#26798 from sarutak/fix-spark-shell-loglevel.

Authored-by: Kousuke Saruta <[email protected]>
Signed-off-by: Marcelo Vanzin <[email protected]>
  • Loading branch information
sarutak authored and Marcelo Vanzin committed Dec 13, 2019
1 parent ec26dde commit 61ebc81
Show file tree
Hide file tree
Showing 3 changed files with 133 additions and 21 deletions.
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)
}
}

0 comments on commit 61ebc81

Please sign in to comment.