Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

add a util method for changing the log level while running #2433

Closed
Show file tree
Hide file tree
Changes from 2 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
6 changes: 6 additions & 0 deletions core/src/main/scala/org/apache/spark/SparkContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -232,6 +232,12 @@ class SparkContext(config: SparkConf) extends Logging {
/** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */
val hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(conf)

/** Control our logLevel
* @param logLevel The desired log level as a string.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would explain that this will override any user-defined log settings.

*/
def setLoggingLevel(logLevel: String) = {
Utils.setLogLevel(org.apache.log4j.Level.toLevel(logLevel))
}
// Optionally log Spark events
private[spark] val eventLogger: Option[EventLoggingListener] = {
if (conf.getBoolean("spark.eventLog.enabled", false)) {
Expand Down
8 changes: 8 additions & 0 deletions core/src/main/scala/org/apache/spark/util/Utils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -1452,6 +1452,14 @@ private[spark] object Utils extends Logging {
}
}

/**
* configure a new log4j level
*/
def setLogLevel(l: org.apache.log4j.Level) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can we call this setLoggingLevel so it's consistent with the other one?

org.apache.log4j.Logger.getRootLogger().setLevel(l)
}


/**
* config a log4j properties used for testsuite
*/
Expand Down
12 changes: 11 additions & 1 deletion core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,9 @@ import com.google.common.base.Charsets
import com.google.common.io.Files
import org.scalatest.FunSuite

class UtilsSuite extends FunSuite {
import org.apache.spark.Logging

class UtilsSuite extends FunSuite with Logging {

test("bytesToString") {
assert(Utils.bytesToString(10) === "10.0 B")
Expand Down Expand Up @@ -297,4 +299,12 @@ class UtilsSuite extends FunSuite {
}
}

// Test for using the util function to change our log levels.
test("log4j log level change") {
Utils.setLogLevel(org.apache.log4j.Level.ALL)
assert(log.isInfoEnabled())
Utils.setLogLevel(org.apache.log4j.Level.ERROR)
assert(!log.isInfoEnabled())
assert(log.isErrorEnabled())
}
}