diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala similarity index 94% rename from core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala rename to core/src/main/scala/org/apache/spark/deploy/Client.scala index 8b066ba1a58326575b91052ac31e878409d136d3..0475bb17c010c7778bbf2c8930fc99b9b01372fe 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -15,22 +15,19 @@ * limitations under the License. */ -package org.apache.spark.deploy.client +package org.apache.spark.deploy import scala.collection.JavaConversions._ import scala.collection.mutable.Map import scala.concurrent._ import akka.actor._ -import akka.actor.Actor +import org.apache.log4j.{Level, Logger} import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.deploy.{Command, DriverDescription} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master import org.apache.spark.util.{AkkaUtils, Utils} -import org.apache.log4j.{Logger, Level} -import akka.remote.RemotingLifecycleEvent /** * Actor that sends a single message to the standalone master and returns the response in the @@ -61,7 +58,7 @@ class DriverActor(master: String, response: Promise[(Boolean, String)]) extends object DriverClient { def main(args: Array[String]) { - val driverArgs = new DriverClientArguments(args) + val driverArgs = new ClientArguments(args) val conf = new SparkConf() if (!driverArgs.logLevel.isGreaterOrEqual(Level.WARN)) { diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala similarity index 96% rename from core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala rename to core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala index 7774a5615cee5b427e1bc64f7d66cdf4c025c4da..50b92e1eabecace5c5223a76755332ec5e1561dc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.deploy.client +package org.apache.spark.deploy import scala.collection.mutable.ListBuffer @@ -24,7 +24,7 @@ import org.apache.log4j.Level /** * Command-line parser for the driver client. */ -private[spark] class DriverClientArguments(args: Array[String]) { +private[spark] class ClientArguments(args: Array[String]) { val defaultCores = 1 val defaultMemory = 512