Suppress log messages in sbt test with two changes:

1. Set akka log level to ERROR before shutting down the actorSystem.
This avoids akka log messages (like Spray) from falling back to INFO
on the Stdout logger
2. Initialize netty to use SLF4J in LocalSparkContext. This ensures that
stack trace thrown during shutdown is handled by SLF4J instead of stdout
This commit is contained in:
Shivaram Venkataraman 2013-07-07 02:30:45 -07:00
parent 3cc6818f13
commit a948f06725
4 changed files with 16 additions and 2 deletions

View file

@ -5,6 +5,7 @@ import serializer.Serializer
import akka.actor.{Actor, ActorRef, Props, ActorSystemImpl, ActorSystem}
import akka.remote.RemoteActorRefProvider
import akka.event.{Logging => AkkaLogging}
import spark.broadcast.BroadcastManager
import spark.storage.BlockManager
@ -51,6 +52,7 @@ class SparkEnv (
broadcastManager.stop()
blockManager.stop()
blockManager.master.stop()
actorSystem.eventStream.setLogLevel(AkkaLogging.ErrorLevel)
actorSystem.shutdown()
// Unfortunately Akka's awaitTermination doesn't actually wait for the Netty server to shut
// down, but let's call it anyway in case it gets fixed in a later release

View file

@ -1,6 +1,7 @@
package spark.deploy
import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated}
import akka.event.{Logging => AkkaLogging}
import spark.deploy.worker.Worker
import spark.deploy.master.Master
@ -43,8 +44,11 @@ class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: I
def stop() {
logInfo("Shutting down local Spark cluster.")
// Stop the workers before the master so they don't get upset that it disconnected
workerActorSystems.foreach(_.eventStream.setLogLevel(AkkaLogging.ErrorLevel))
workerActorSystems.foreach(_.shutdown())
workerActorSystems.foreach(_.awaitTermination())
masterActorSystems.foreach(_.eventStream.setLogLevel(AkkaLogging.ErrorLevel))
masterActorSystems.foreach(_.shutdown())
masterActorSystems.foreach(_.awaitTermination())
}

View file

@ -85,7 +85,6 @@ class FileServerSuite extends FunSuite with LocalSparkContext {
in.close()
_ * fileVal + _ * fileVal
}.collect
println(result)
assert(result.toSet === Set((1,200), (2,300), (3,500)))
}

View file

@ -2,12 +2,21 @@ package spark
import org.scalatest.Suite
import org.scalatest.BeforeAndAfterEach
import org.scalatest.BeforeAndAfterAll
import org.jboss.netty.logging.InternalLoggerFactory
import org.jboss.netty.logging.Slf4JLoggerFactory
/** Manages a local `sc` {@link SparkContext} variable, correctly stopping it after each test. */
trait LocalSparkContext extends BeforeAndAfterEach { self: Suite =>
trait LocalSparkContext extends BeforeAndAfterEach with BeforeAndAfterAll { self: Suite =>
@transient var sc: SparkContext = _
override def beforeAll() {
InternalLoggerFactory.setDefaultFactory(new Slf4JLoggerFactory());
super.beforeAll()
}
override def afterEach() {
resetSparkContext()
super.afterEach()