diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala
index 16b00d15aab2e5ae1c3a69ab40d040f2ece3e787..ec59b4f48fddd8bb6887dc4bfdaef52c720ca311 100644
--- a/core/src/main/scala/spark/SparkEnv.scala
+++ b/core/src/main/scala/spark/SparkEnv.scala
@@ -5,7 +5,6 @@ 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
@@ -52,7 +51,6 @@ 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 cb85419ae43b560e6fb4f049ec3b9ba31c8068ed..939f26b6f48423ae0e34052c88da832157531823 100644
--- a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala
+++ b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala
@@ -1,7 +1,6 @@
 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
@@ -44,11 +43,9 @@ 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())
   }