aboutsummaryrefslogtreecommitdiff
path: root/core/src/main/scala/org/apache
diff options
context:
space:
mode:
Diffstat (limited to 'core/src/main/scala/org/apache')
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala4
2 files changed, 5 insertions, 5 deletions
diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala
index 66e1e64500..9b31497adf 100644
--- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala
@@ -50,7 +50,7 @@ private[mesos] class MesosClusterDispatcher(
extends Logging {
private val publicAddress = Option(conf.getenv("SPARK_PUBLIC_DNS")).getOrElse(args.host)
- private val recoveryMode = conf.get("spark.mesos.deploy.recoveryMode", "NONE").toUpperCase()
+ private val recoveryMode = conf.get("spark.deploy.recoveryMode", "NONE").toUpperCase()
logInfo("Recovery mode in Mesos dispatcher set to: " + recoveryMode)
private val engineFactory = recoveryMode match {
@@ -98,8 +98,8 @@ private[mesos] object MesosClusterDispatcher extends Logging {
conf.setMaster(dispatcherArgs.masterUrl)
conf.setAppName(dispatcherArgs.name)
dispatcherArgs.zookeeperUrl.foreach { z =>
- conf.set("spark.mesos.deploy.recoveryMode", "ZOOKEEPER")
- conf.set("spark.mesos.deploy.zookeeper.url", z)
+ conf.set("spark.deploy.recoveryMode", "ZOOKEEPER")
+ conf.set("spark.deploy.zookeeper.url", z)
}
val dispatcher = new MesosClusterDispatcher(dispatcherArgs, conf)
dispatcher.start()
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala
index e0c547dce6..092d9e4182 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala
@@ -53,9 +53,9 @@ private[spark] trait MesosClusterPersistenceEngine {
* all of them reuses the same connection pool.
*/
private[spark] class ZookeeperMesosClusterPersistenceEngineFactory(conf: SparkConf)
- extends MesosClusterPersistenceEngineFactory(conf) {
+ extends MesosClusterPersistenceEngineFactory(conf) with Logging {
- lazy val zk = SparkCuratorUtil.newClient(conf, "spark.mesos.deploy.zookeeper.url")
+ lazy val zk = SparkCuratorUtil.newClient(conf)
def createEngine(path: String): MesosClusterPersistenceEngine = {
new ZookeeperMesosClusterPersistenceEngine(path, zk, conf)