Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-5265

Submitting applications on Standalone cluster controlled by Zookeeper forces to know active master

    XMLWordPrintableJSON

Details

    Description

      Hi, this is my first JIRA here, so I hope it is clear enough.

      I'm using Spark 1.2.0 and trying to submit an application on a Spark Standalone cluster in cluster deploy mode with supervise.

      Standalone cluster is running in high availability mode, using Zookeeper to provide leader election between three available Masters (named master1, master2 and master3).

      As read at Spark's documentation, to register a Worker to the Standalone cluster, I provide complete cluster info as the spark route.
      I mean, spark://master1:7077,master2:7077,master3:7077
      and that route is parsed and three attempts are launched, first one to master1:7077, second one to master2:7077 and third one to master3:7077.
      This works great!

      But if I try to do the same while submitting applications, it fails.
      I mean, if I provide complete cluster info as the --master option to spark-submit script, it throws an exception because it tries to connect as it was a single node.
      Example:
      spark-submit --class org.apache.spark.examples.SparkPi --master spark://master1:7077,master2:7077,master3:7077 --deploy-mode cluster --supervise examples.jar 100

      This is the output I got:
      Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties
      15/01/14 17:02:11 INFO SecurityManager: Changing view acls to: mytest
      15/01/14 17:02:11 INFO SecurityManager: Changing modify acls to: mytest
      15/01/14 17:02:11 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users with view permissions: Set(mytest); users with modify permissions: Set(mytest)
      15/01/14 17:02:11 INFO Slf4jLogger: Slf4jLogger started
      15/01/14 17:02:11 INFO Utils: Successfully started service 'driverClient' on port 53930.
      15/01/14 17:02:11 ERROR OneForOneStrategy: Invalid master URL: spark://master1:7077,master2:7077,master3:7077
      akka.actor.ActorInitializationException: exception during creation
      at akka.actor.ActorInitializationException$.apply(Actor.scala:164)
      at akka.actor.ActorCell.create(ActorCell.scala:596)
      at akka.actor.ActorCell.invokeAll$1(ActorCell.scala:456)
      at akka.actor.ActorCell.systemInvoke(ActorCell.scala:478)
      at akka.dispatch.Mailbox.processAllSystemMessages(Mailbox.scala:263)
      at akka.dispatch.Mailbox.run(Mailbox.scala:219)
      at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:393)
      at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
      at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
      at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
      at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
      Caused by: org.apache.spark.SparkException: Invalid master URL: spark://master1:7077,master2:7077,master3:7077
      at org.apache.spark.deploy.master.Master$.toAkkaUrl(Master.scala:830)
      at org.apache.spark.deploy.ClientActor.preStart(Client.scala:42)
      at akka.actor.Actor$class.aroundPreStart(Actor.scala:470)
      at org.apache.spark.deploy.ClientActor.aroundPreStart(Client.scala:35)
      at akka.actor.ActorCell.create(ActorCell.scala:580)
      ... 9 more

      Shouldn't it parse it as on Worker registration?
      It will not force client to know which is the current active Master of the Standalone cluster.

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              zujorv Roque Vassal'lo
              Votes:
              1 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: