diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala
index a185954089528103e05f28fcb6fd72feda61457c..b0665570e2681b216e664dfe2111afbe4ef96d6e 100644
--- a/core/src/main/scala/org/apache/spark/SparkEnv.scala
+++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala
@@ -20,6 +20,8 @@ package org.apache.spark
 import java.io.File
 import java.net.Socket
 
+import akka.actor.ActorSystem
+
 import scala.collection.JavaConversions._
 import scala.collection.mutable
 import scala.util.Properties
@@ -75,7 +77,8 @@ class SparkEnv (
     val conf: SparkConf) extends Logging {
 
   // TODO Remove actorSystem
-  val actorSystem = rpcEnv.asInstanceOf[AkkaRpcEnv].actorSystem
+  @deprecated("Actor system is no longer supported as of 1.4")
+  val actorSystem: ActorSystem = rpcEnv.asInstanceOf[AkkaRpcEnv].actorSystem
 
   private[spark] var isStopped = false
   private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]()