Skip to content

Commit

Permalink
For SPARK-1082, Use Curator for ZK interaction in standalone cluster
Browse files Browse the repository at this point in the history
Author: Raymond Liu <raymond.liu@intel.com>

Closes #611 from colorant/curator and squashes the following commits:

7556aa1 [Raymond Liu] Address review comments
af92e1f [Raymond Liu] Fix coding style
964f3c2 [Raymond Liu] Ignore NodeExists exception
6df2966 [Raymond Liu] Rewrite zookeeper client code with curator
  • Loading branch information
colorant authored and aarondav committed Feb 25, 2014
1 parent 1f4c7f7 commit c852201
Show file tree
Hide file tree
Showing 9 changed files with 99 additions and 300 deletions.
4 changes: 2 additions & 2 deletions core/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -55,8 +55,8 @@
<artifactId>avro-ipc</artifactId>
</dependency>
<dependency>
<groupId>org.apache.zookeeper</groupId>
<artifactId>zookeeper</artifactId>
<groupId>org.apache.curator</groupId>
<artifactId>curator-recipes</artifactId>
</dependency>
<dependency>
<groupId>org.eclipse.jetty</groupId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import org.apache.spark.deploy.master.MasterMessages.ElectedLeader
* [[org.apache.spark.deploy.master.MasterMessages.RevokedLeadership RevokedLeadership]]
*/
private[spark] trait LeaderElectionAgent extends Actor {
//TODO: LeaderElectionAgent does not necessary to be an Actor anymore, need refactoring.
val masterActor: ActorRef
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,10 +28,6 @@ private[master] object MasterMessages {

case object RevokedLeadership

// Actor System to LeaderElectionAgent

case object CheckLeader

// Actor System to Master

case object CheckForWorkerTimeOut
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,53 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.deploy.master

import org.apache.spark.{SparkConf, Logging}
import org.apache.curator.framework.{CuratorFramework, CuratorFrameworkFactory}
import org.apache.curator.retry.ExponentialBackoffRetry
import org.apache.zookeeper.KeeperException


object SparkCuratorUtil extends Logging {

val ZK_CONNECTION_TIMEOUT_MILLIS = 15000
val ZK_SESSION_TIMEOUT_MILLIS = 60000
val RETRY_WAIT_MILLIS = 5000
val MAX_RECONNECT_ATTEMPTS = 3

def newClient(conf: SparkConf): CuratorFramework = {
val ZK_URL = conf.get("spark.deploy.zookeeper.url")
val zk = CuratorFrameworkFactory.newClient(ZK_URL,
ZK_SESSION_TIMEOUT_MILLIS, ZK_CONNECTION_TIMEOUT_MILLIS,
new ExponentialBackoffRetry(RETRY_WAIT_MILLIS, MAX_RECONNECT_ATTEMPTS))
zk.start()
zk
}

def mkdir(zk: CuratorFramework, path: String) {
if (zk.checkExists().forPath(path) == null) {
try {
zk.create().creatingParentsIfNeeded().forPath(path)
} catch {
case nodeExist: KeeperException.NodeExistsException =>
// do nothing, ignore node existing exception.
case e: Exception => throw e
}
}
}
}

This file was deleted.

Loading

0 comments on commit c852201

Please sign in to comment.