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
5 changes: 5 additions & 0 deletions conf/log4j.properties.template
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,11 @@ 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 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
log4j.logger.org.spark-project.jetty=WARN
log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR
Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,11 @@ 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 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
log4j.logger.org.spark-project.jetty=WARN
log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR
Expand Down
45 changes: 23 additions & 22 deletions core/src/main/scala/org/apache/spark/Logging.scala
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

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.impl.StaticLoggerBinder

Expand Down Expand Up @@ -119,30 +119,31 @@ trait Logging {
val usingLog4j12 = "org.slf4j.impl.Log4jLoggerFactory".equals(binderClass)
if (usingLog4j12) {
val log4j12Initialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements
// scalastyle:off println
if (!log4j12Initialized) {
// scalastyle:off println
if (Utils.isInInterpreter) {
val replDefaultLogProps = "org/apache/spark/log4j-defaults-repl.properties"
Option(Utils.getSparkClassLoader.getResource(replDefaultLogProps)) match {
case Some(url) =>
PropertyConfigurator.configure(url)
System.err.println(s"Using Spark's repl log4j profile: $replDefaultLogProps")
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")
}
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

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -123,18 +123,19 @@ private[repl] trait SparkILoopInit {
def initializeSpark() {
intp.beQuietDuring {
command("""
@transient val sc = {
val _sc = org.apache.spark.repl.Main.interp.createSparkContext()
println("Spark context available as sc.")
_sc
}
@transient val sc = {
val _sc = org.apache.spark.repl.Main.interp.createSparkContext()
println("Spark context available as sc " +
s"(master = ${_sc.master}, app id = ${_sc.applicationId}).")
_sc
}
""")
command("""
@transient val sqlContext = {
val _sqlContext = org.apache.spark.repl.Main.interp.createSQLContext()
println("SQL context available as sqlContext.")
_sqlContext
}
@transient val sqlContext = {
val _sqlContext = org.apache.spark.repl.Main.interp.createSQLContext()
println("SQL context available as sqlContext.")
_sqlContext
}
""")
command("import org.apache.spark.SparkContext._")
command("import sqlContext.implicits._")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,18 +37,19 @@ class SparkILoop(in0: Option[BufferedReader], out: JPrintWriter)
def initializeSpark() {
intp.beQuietDuring {
processLine("""
@transient val sc = {
val _sc = org.apache.spark.repl.Main.createSparkContext()
println("Spark context available as sc.")
_sc
}
@transient val sc = {
val _sc = org.apache.spark.repl.Main.createSparkContext()
println("Spark context available as sc " +
s"(master = ${_sc.master}, app id = ${_sc.applicationId}).")
_sc
}
""")
processLine("""
@transient val sqlContext = {
val _sqlContext = org.apache.spark.repl.Main.createSQLContext()
println("SQL context available as sqlContext.")
_sqlContext
}
@transient val sqlContext = {
val _sqlContext = org.apache.spark.repl.Main.createSQLContext()
println("SQL context available as sqlContext.")
_sqlContext
}
""")
processLine("import org.apache.spark.SparkContext._")
processLine("import sqlContext.implicits._")
Expand Down Expand Up @@ -85,7 +86,7 @@ class SparkILoop(in0: Option[BufferedReader], out: JPrintWriter)
/** Available commands */
override def commands: List[LoopCommand] = sparkStandardCommands

/**
/**
* 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
* hack, but there isn't another hook available to us at this point.
Expand All @@ -98,7 +99,7 @@ class SparkILoop(in0: Option[BufferedReader], out: JPrintWriter)

object SparkILoop {

/**
/**
* Creates an interpreter loop with default settings and feeds
* the given code to it as input.
*/
Expand Down