[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 <vanzin@cloudera.com>

Closes #9816 from vanzin/shell-logging.
This commit is contained in:
Marcelo Vanzin 2015-11-24 15:08:02 -06:00 committed by Imran Rashid
parent f315272279
commit e6dd237463
6 changed files with 57 additions and 77 deletions

View file

@ -22,6 +22,11 @@ log4j.appender.console.target=System.err
log4j.appender.console.layout=org.apache.log4j.PatternLayout 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 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 # Settings to quiet third party logs that are too verbose
log4j.logger.org.spark-project.jetty=WARN log4j.logger.org.spark-project.jetty=WARN
log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR

View file

@ -1,33 +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=WARN, 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
# Settings to quiet third party logs that are too verbose
log4j.logger.org.spark-project.jetty=WARN
log4j.logger.org.spark-project.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

View file

@ -22,6 +22,11 @@ log4j.appender.console.target=System.err
log4j.appender.console.layout=org.apache.log4j.PatternLayout 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 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 # Settings to quiet third party logs that are too verbose
log4j.logger.org.spark-project.jetty=WARN log4j.logger.org.spark-project.jetty=WARN
log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR

View file

@ -17,7 +17,7 @@
package org.apache.spark package org.apache.spark
import org.apache.log4j.{LogManager, PropertyConfigurator} import org.apache.log4j.{Level, LogManager, PropertyConfigurator}
import org.slf4j.{Logger, LoggerFactory} import org.slf4j.{Logger, LoggerFactory}
import org.slf4j.impl.StaticLoggerBinder import org.slf4j.impl.StaticLoggerBinder
@ -119,30 +119,31 @@ trait Logging {
val usingLog4j12 = "org.slf4j.impl.Log4jLoggerFactory".equals(binderClass) val usingLog4j12 = "org.slf4j.impl.Log4jLoggerFactory".equals(binderClass)
if (usingLog4j12) { if (usingLog4j12) {
val log4j12Initialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements val log4j12Initialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements
// scalastyle:off println
if (!log4j12Initialized) { if (!log4j12Initialized) {
// scalastyle:off println val defaultLogProps = "org/apache/spark/log4j-defaults.properties"
if (Utils.isInInterpreter) { Option(Utils.getSparkClassLoader.getResource(defaultLogProps)) match {
val replDefaultLogProps = "org/apache/spark/log4j-defaults-repl.properties" case Some(url) =>
Option(Utils.getSparkClassLoader.getResource(replDefaultLogProps)) match { PropertyConfigurator.configure(url)
case Some(url) => System.err.println(s"Using Spark's default log4j profile: $defaultLogProps")
PropertyConfigurator.configure(url) case None =>
System.err.println(s"Using Spark's repl log4j profile: $replDefaultLogProps") System.err.println(s"Spark was unable to load $defaultLogProps")
System.err.println("To adjust logging level use sc.setLogLevel(\"INFO\")")
case None =>
System.err.println(s"Spark was unable to load $replDefaultLogProps")
}
} else {
val defaultLogProps = "org/apache/spark/log4j-defaults.properties"
Option(Utils.getSparkClassLoader.getResource(defaultLogProps)) match {
case Some(url) =>
PropertyConfigurator.configure(url)
System.err.println(s"Using Spark's default log4j profile: $defaultLogProps")
case None =>
System.err.println(s"Spark was unable to load $defaultLogProps")
}
} }
// scalastyle:on println
} }
if (Utils.isInInterpreter) {
// Use the repl's main class to define the default log level when running the shell,
// overriding the root logger's config if they're different.
val rootLogger = LogManager.getRootLogger()
val replLogger = LogManager.getLogger("org.apache.spark.repl.Main")
val replLevel = Option(replLogger.getLevel()).getOrElse(Level.WARN)
if (replLevel != rootLogger.getEffectiveLevel()) {
System.err.printf("Setting default log level to \"%s\".\n", replLevel)
System.err.println("To adjust logging level use sc.setLogLevel(newLevel).")
rootLogger.setLevel(replLevel)
}
}
// scalastyle:on println
} }
Logging.initialized = true Logging.initialized = true

View file

@ -123,18 +123,19 @@ private[repl] trait SparkILoopInit {
def initializeSpark() { def initializeSpark() {
intp.beQuietDuring { intp.beQuietDuring {
command(""" command("""
@transient val sc = { @transient val sc = {
val _sc = org.apache.spark.repl.Main.interp.createSparkContext() val _sc = org.apache.spark.repl.Main.interp.createSparkContext()
println("Spark context available as sc.") println("Spark context available as sc " +
_sc s"(master = ${_sc.master}, app id = ${_sc.applicationId}).")
} _sc
}
""") """)
command(""" command("""
@transient val sqlContext = { @transient val sqlContext = {
val _sqlContext = org.apache.spark.repl.Main.interp.createSQLContext() val _sqlContext = org.apache.spark.repl.Main.interp.createSQLContext()
println("SQL context available as sqlContext.") println("SQL context available as sqlContext.")
_sqlContext _sqlContext
} }
""") """)
command("import org.apache.spark.SparkContext._") command("import org.apache.spark.SparkContext._")
command("import sqlContext.implicits._") command("import sqlContext.implicits._")

View file

@ -37,18 +37,19 @@ class SparkILoop(in0: Option[BufferedReader], out: JPrintWriter)
def initializeSpark() { def initializeSpark() {
intp.beQuietDuring { intp.beQuietDuring {
processLine(""" processLine("""
@transient val sc = { @transient val sc = {
val _sc = org.apache.spark.repl.Main.createSparkContext() val _sc = org.apache.spark.repl.Main.createSparkContext()
println("Spark context available as sc.") println("Spark context available as sc " +
_sc s"(master = ${_sc.master}, app id = ${_sc.applicationId}).")
} _sc
}
""") """)
processLine(""" processLine("""
@transient val sqlContext = { @transient val sqlContext = {
val _sqlContext = org.apache.spark.repl.Main.createSQLContext() val _sqlContext = org.apache.spark.repl.Main.createSQLContext()
println("SQL context available as sqlContext.") println("SQL context available as sqlContext.")
_sqlContext _sqlContext
} }
""") """)
processLine("import org.apache.spark.SparkContext._") processLine("import org.apache.spark.SparkContext._")
processLine("import sqlContext.implicits._") processLine("import sqlContext.implicits._")
@ -85,7 +86,7 @@ class SparkILoop(in0: Option[BufferedReader], out: JPrintWriter)
/** Available commands */ /** Available commands */
override def commands: List[LoopCommand] = sparkStandardCommands override def commands: List[LoopCommand] = sparkStandardCommands
/** /**
* We override `loadFiles` because we need to initialize Spark *before* the REPL * We override `loadFiles` because we need to initialize Spark *before* the REPL
* sees any files, so that the Spark context is visible in those files. This is a bit of a * sees any files, so that the Spark context is visible in those files. This is a bit of a
* hack, but there isn't another hook available to us at this point. * hack, but there isn't another hook available to us at this point.
@ -98,7 +99,7 @@ class SparkILoop(in0: Option[BufferedReader], out: JPrintWriter)
object SparkILoop { object SparkILoop {
/** /**
* Creates an interpreter loop with default settings and feeds * Creates an interpreter loop with default settings and feeds
* the given code to it as input. * the given code to it as input.
*/ */