From a948f0672541bd68be020f07134659aee2f38403 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman <shivaram@eecs.berkeley.edu> Date: Sun, 7 Jul 2013 02:30:45 -0700 Subject: [PATCH] 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 --- core/src/main/scala/spark/SparkEnv.scala | 2 ++ .../main/scala/spark/deploy/LocalSparkCluster.scala | 4 ++++ core/src/test/scala/spark/FileServerSuite.scala | 1 - core/src/test/scala/spark/LocalSparkContext.scala | 11 ++++++++++- 4 files changed, 16 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index ec59b4f48f..16b00d15aa 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -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 diff --git a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala index 55bb61b0cc..cb85419ae4 100644 --- a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala @@ -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()) } diff --git a/core/src/test/scala/spark/FileServerSuite.scala b/core/src/test/scala/spark/FileServerSuite.scala index f1a35bced3..9c24ca430d 100644 --- a/core/src/test/scala/spark/FileServerSuite.scala +++ b/core/src/test/scala/spark/FileServerSuite.scala @@ -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))) } diff --git a/core/src/test/scala/spark/LocalSparkContext.scala b/core/src/test/scala/spark/LocalSparkContext.scala index 76d5258b02..bd184222ed 100644 --- a/core/src/test/scala/spark/LocalSparkContext.scala +++ b/core/src/test/scala/spark/LocalSparkContext.scala @@ -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() -- GitLab