diff --git a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
similarity index 95%
rename from core/src/main/scala/org/apache/spark/deploy/client/Client.scala
rename to core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
index 953755e40dbbf0edd10285c0c574c411a9885205..c5a0d1fd018b43f7a34804f20b528497575f27dd 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
@@ -34,16 +34,17 @@ import org.apache.spark.util.AkkaUtils
 
 
 /**
- * The main class used to talk to a Spark deploy cluster. Takes a master URL, an app description,
- * and a listener for cluster events, and calls back the listener when various events occur.
+ * Interface allowing applications to speak with a Spark deploy cluster. Takes a master URL,
+ * an app description, and a listener for cluster events, and calls back the listener when various
+ * events occur.
  *
  * @param masterUrls Each url should look like spark://host:port.
  */
-private[spark] class Client(
+private[spark] class AppClient(
     actorSystem: ActorSystem,
     masterUrls: Array[String],
     appDescription: ApplicationDescription,
-    listener: ClientListener)
+    listener: AppClientListener)
   extends Logging {
 
   val REGISTRATION_TIMEOUT = 20.seconds
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/ClientListener.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala
similarity index 97%
rename from core/src/main/scala/org/apache/spark/deploy/client/ClientListener.scala
rename to core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala
index be7a11bd1553724376d334245bf14a30d6a06a35..55d4ef1b31aaacbe49878ba4bb3cabebfc7946a2 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/ClientListener.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala
@@ -24,7 +24,7 @@ package org.apache.spark.deploy.client
  *
  * Users of this API should *not* block inside the callback methods.
  */
-private[spark] trait ClientListener {
+private[spark] trait AppClientListener {
   def connected(appId: String): Unit
 
   /** Disconnection may be a temporary state, as we fail over to a new Master. */
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
index 5b62d3ba6c0e0ac8bb456ea8a8d39d69f3117165..9359bf1a4b6723eaa3875cd6698ab01ebaa6e712 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
@@ -23,7 +23,7 @@ import org.apache.spark.deploy.{Command, ApplicationDescription}
 
 private[spark] object TestClient {
 
-  class TestListener extends ClientListener with Logging {
+  class TestListener extends AppClientListener with Logging {
     def connected(id: String) {
       logInfo("Connected to master, got app ID " + id)
     }
@@ -49,7 +49,7 @@ private[spark] object TestClient {
     val desc = new ApplicationDescription(
       "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home", "ignored")
     val listener = new TestListener
-    val client = new Client(actorSystem, Array(url), desc, listener)
+    val client = new AppClient(actorSystem, Array(url), desc, listener)
     client.start()
     actorSystem.awaitTermination()
   }
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
index 404ce7a452c60ec552e483d471b2f9d633f26995..4da49c0735f9660648cd24ec6e5b085d1c2e72b5 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
@@ -20,7 +20,7 @@ package org.apache.spark.scheduler.cluster
 import scala.collection.mutable.HashMap
 
 import org.apache.spark.{Logging, SparkContext}
-import org.apache.spark.deploy.client.{Client, ClientListener}
+import org.apache.spark.deploy.client.{AppClient, AppClientListener}
 import org.apache.spark.deploy.{Command, ApplicationDescription}
 import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SlaveLost, TaskSchedulerImpl}
 import org.apache.spark.util.Utils
@@ -31,10 +31,10 @@ private[spark] class SparkDeploySchedulerBackend(
     masters: Array[String],
     appName: String)
   extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem)
-  with ClientListener
+  with AppClientListener
   with Logging {
 
-  var client: Client = null
+  var client: AppClient = null
   var stopping = false
   var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _
 
@@ -54,7 +54,7 @@ private[spark] class SparkDeploySchedulerBackend(
     val appDesc = new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome,
         "http://" + sc.ui.appUIAddress)
 
-    client = new Client(sc.env.actorSystem, masters, appDesc, this)
+    client = new AppClient(sc.env.actorSystem, masters, appDesc, this)
     client.start()
   }