Skip to content

Commit e6dd237

Browse files
Marcelo Vanzinsquito
Marcelo Vanzin
authored andcommitted
[SPARK-11929][CORE] Make the repl log4j configuration override the root logger.
In the default Spark distribution, there are currently two separate log4j config files, with different default values for the root logger, so that when running the shell you have a different default log level. This makes the shell more usable, since the logs don't overwhelm the output. But if you install a custom log4j.properties, you lose that, because then it's going to be used no matter whether you're running a regular app or the shell. With this change, the overriding of the log level is done differently; the log level repl's main class (org.apache.spark.repl.Main) is used to define the root logger's level when running the shell, defaulting to WARN if it's not set explicitly. On a somewhat related change, the shell output about the "sc" variable was changed a bit to contain a little more useful information about the application, since when the root logger's log level is WARN, that information is never shown to the user. Author: Marcelo Vanzin <[email protected]> Closes #9816 from vanzin/shell-logging.
1 parent f315272 commit e6dd237

File tree

6 files changed

+57
-77
lines changed

6 files changed

+57
-77
lines changed

conf/log4j.properties.template

+5
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,11 @@ log4j.appender.console.target=System.err
2222
log4j.appender.console.layout=org.apache.log4j.PatternLayout
2323
log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n
2424

25+
# Set the default spark-shell log level to WARN. When running the spark-shell, the
26+
# log level for this class is used to overwrite the root logger's log level, so that
27+
# the user can have different defaults for the shell and regular Spark apps.
28+
log4j.logger.org.apache.spark.repl.Main=WARN
29+
2530
# Settings to quiet third party logs that are too verbose
2631
log4j.logger.org.spark-project.jetty=WARN
2732
log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR

core/src/main/resources/org/apache/spark/log4j-defaults-repl.properties

-33
This file was deleted.

core/src/main/resources/org/apache/spark/log4j-defaults.properties

+5
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,11 @@ log4j.appender.console.target=System.err
2222
log4j.appender.console.layout=org.apache.log4j.PatternLayout
2323
log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n
2424

25+
# Set the default spark-shell log level to WARN. When running the spark-shell, the
26+
# log level for this class is used to overwrite the root logger's log level, so that
27+
# the user can have different defaults for the shell and regular Spark apps.
28+
log4j.logger.org.apache.spark.repl.Main=WARN
29+
2530
# Settings to quiet third party logs that are too verbose
2631
log4j.logger.org.spark-project.jetty=WARN
2732
log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR

core/src/main/scala/org/apache/spark/Logging.scala

+23-22
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@
1717

1818
package org.apache.spark
1919

20-
import org.apache.log4j.{LogManager, PropertyConfigurator}
20+
import org.apache.log4j.{Level, LogManager, PropertyConfigurator}
2121
import org.slf4j.{Logger, LoggerFactory}
2222
import org.slf4j.impl.StaticLoggerBinder
2323

@@ -119,30 +119,31 @@ trait Logging {
119119
val usingLog4j12 = "org.slf4j.impl.Log4jLoggerFactory".equals(binderClass)
120120
if (usingLog4j12) {
121121
val log4j12Initialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements
122+
// scalastyle:off println
122123
if (!log4j12Initialized) {
123-
// scalastyle:off println
124-
if (Utils.isInInterpreter) {
125-
val replDefaultLogProps = "org/apache/spark/log4j-defaults-repl.properties"
126-
Option(Utils.getSparkClassLoader.getResource(replDefaultLogProps)) match {
127-
case Some(url) =>
128-
PropertyConfigurator.configure(url)
129-
System.err.println(s"Using Spark's repl log4j profile: $replDefaultLogProps")
130-
System.err.println("To adjust logging level use sc.setLogLevel(\"INFO\")")
131-
case None =>
132-
System.err.println(s"Spark was unable to load $replDefaultLogProps")
133-
}
134-
} else {
135-
val defaultLogProps = "org/apache/spark/log4j-defaults.properties"
136-
Option(Utils.getSparkClassLoader.getResource(defaultLogProps)) match {
137-
case Some(url) =>
138-
PropertyConfigurator.configure(url)
139-
System.err.println(s"Using Spark's default log4j profile: $defaultLogProps")
140-
case None =>
141-
System.err.println(s"Spark was unable to load $defaultLogProps")
142-
}
124+
val defaultLogProps = "org/apache/spark/log4j-defaults.properties"
125+
Option(Utils.getSparkClassLoader.getResource(defaultLogProps)) match {
126+
case Some(url) =>
127+
PropertyConfigurator.configure(url)
128+
System.err.println(s"Using Spark's default log4j profile: $defaultLogProps")
129+
case None =>
130+
System.err.println(s"Spark was unable to load $defaultLogProps")
143131
}
144-
// scalastyle:on println
145132
}
133+
134+
if (Utils.isInInterpreter) {
135+
// Use the repl's main class to define the default log level when running the shell,
136+
// overriding the root logger's config if they're different.
137+
val rootLogger = LogManager.getRootLogger()
138+
val replLogger = LogManager.getLogger("org.apache.spark.repl.Main")
139+
val replLevel = Option(replLogger.getLevel()).getOrElse(Level.WARN)
140+
if (replLevel != rootLogger.getEffectiveLevel()) {
141+
System.err.printf("Setting default log level to \"%s\".\n", replLevel)
142+
System.err.println("To adjust logging level use sc.setLogLevel(newLevel).")
143+
rootLogger.setLevel(replLevel)
144+
}
145+
}
146+
// scalastyle:on println
146147
}
147148
Logging.initialized = true
148149

repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala

+11-10
Original file line numberDiff line numberDiff line change
@@ -123,18 +123,19 @@ private[repl] trait SparkILoopInit {
123123
def initializeSpark() {
124124
intp.beQuietDuring {
125125
command("""
126-
@transient val sc = {
127-
val _sc = org.apache.spark.repl.Main.interp.createSparkContext()
128-
println("Spark context available as sc.")
129-
_sc
130-
}
126+
@transient val sc = {
127+
val _sc = org.apache.spark.repl.Main.interp.createSparkContext()
128+
println("Spark context available as sc " +
129+
s"(master = ${_sc.master}, app id = ${_sc.applicationId}).")
130+
_sc
131+
}
131132
""")
132133
command("""
133-
@transient val sqlContext = {
134-
val _sqlContext = org.apache.spark.repl.Main.interp.createSQLContext()
135-
println("SQL context available as sqlContext.")
136-
_sqlContext
137-
}
134+
@transient val sqlContext = {
135+
val _sqlContext = org.apache.spark.repl.Main.interp.createSQLContext()
136+
println("SQL context available as sqlContext.")
137+
_sqlContext
138+
}
138139
""")
139140
command("import org.apache.spark.SparkContext._")
140141
command("import sqlContext.implicits._")

repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala

+13-12
Original file line numberDiff line numberDiff line change
@@ -37,18 +37,19 @@ class SparkILoop(in0: Option[BufferedReader], out: JPrintWriter)
3737
def initializeSpark() {
3838
intp.beQuietDuring {
3939
processLine("""
40-
@transient val sc = {
41-
val _sc = org.apache.spark.repl.Main.createSparkContext()
42-
println("Spark context available as sc.")
43-
_sc
44-
}
40+
@transient val sc = {
41+
val _sc = org.apache.spark.repl.Main.createSparkContext()
42+
println("Spark context available as sc " +
43+
s"(master = ${_sc.master}, app id = ${_sc.applicationId}).")
44+
_sc
45+
}
4546
""")
4647
processLine("""
47-
@transient val sqlContext = {
48-
val _sqlContext = org.apache.spark.repl.Main.createSQLContext()
49-
println("SQL context available as sqlContext.")
50-
_sqlContext
51-
}
48+
@transient val sqlContext = {
49+
val _sqlContext = org.apache.spark.repl.Main.createSQLContext()
50+
println("SQL context available as sqlContext.")
51+
_sqlContext
52+
}
5253
""")
5354
processLine("import org.apache.spark.SparkContext._")
5455
processLine("import sqlContext.implicits._")
@@ -85,7 +86,7 @@ class SparkILoop(in0: Option[BufferedReader], out: JPrintWriter)
8586
/** Available commands */
8687
override def commands: List[LoopCommand] = sparkStandardCommands
8788

88-
/**
89+
/**
8990
* We override `loadFiles` because we need to initialize Spark *before* the REPL
9091
* sees any files, so that the Spark context is visible in those files. This is a bit of a
9192
* hack, but there isn't another hook available to us at this point.
@@ -98,7 +99,7 @@ class SparkILoop(in0: Option[BufferedReader], out: JPrintWriter)
9899

99100
object SparkILoop {
100101

101-
/**
102+
/**
102103
* Creates an interpreter loop with default settings and feeds
103104
* the given code to it as input.
104105
*/

0 commit comments

Comments
 (0)