Repository: giraph
Updated Branches:
  refs/heads/trunk 63b8c412f -> 226fa4bb4


GIRAPH-992 Zookeeper logs have too many NodeExists

https://phabricator.fb.com/D1827310


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/226fa4bb
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/226fa4bb
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/226fa4bb

Branch: refs/heads/trunk
Commit: 226fa4bb47a97ac825e05274cba77914f9c96252
Parents: 63b8c41
Author: Sergey Edunov <[email protected]>
Authored: Wed Feb 4 17:26:55 2015 -0800
Committer: Sergey Edunov <[email protected]>
Committed: Thu Feb 5 10:06:08 2015 -0800

----------------------------------------------------------------------
 CHANGELOG                                                |  4 +++-
 .../java/org/apache/giraph/master/BspServiceMaster.java  | 11 +++++++++++
 .../src/main/java/org/apache/giraph/zk/ZooKeeperExt.java |  7 +++++--
 3 files changed, 19 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/226fa4bb/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index 8da1efe..b1817b6 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,7 +1,9 @@
 Giraph Change Log
 
 Release 1.2.0 - unreleased
-  GIRAPH-991: Update versions of guava and swift (edunov)
+  GIRAPH-992: Zookeeper logs have too many NodeExists (edunov)
+
+  GIRAPH-991: Update version of swift (edunov)
 
   GIRAPH-987: Improve naming for ReduceOperation (ikabiljo via majakabiljo)
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/226fa4bb/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
----------------------------------------------------------------------
diff --git 
a/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java 
b/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
index 5281ae8..e942157 100644
--- a/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
+++ b/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
@@ -678,6 +678,17 @@ public class BspServiceMaster<I extends WritableComparable,
       LOG.info(logPrefix + ": Starting to write input split data " +
           "to zookeeper with " + inputSplitThreadCount + " threads");
     }
+    try {
+      getZkExt().createExt(inputSplitsPath, null,
+          Ids.OPEN_ACL_UNSAFE,
+          CreateMode.PERSISTENT,
+          false);
+    } catch (KeeperException e) {
+      LOG.info(logPrefix + ": Node " +
+          inputSplitsPath + " keeper exception " + e);
+    } catch (InterruptedException e) {
+      throw new IllegalStateException(logPrefix + ' ' + e.getMessage(), e);
+    }
     ExecutorService taskExecutor =
         Executors.newFixedThreadPool(inputSplitThreadCount);
     boolean writeLocations = USE_INPUT_SPLIT_LOCALITY.get(conf);

http://git-wip-us.apache.org/repos/asf/giraph/blob/226fa4bb/giraph-core/src/main/java/org/apache/giraph/zk/ZooKeeperExt.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/zk/ZooKeeperExt.java 
b/giraph-core/src/main/java/org/apache/giraph/zk/ZooKeeperExt.java
index 4044fed..1eb4c8b 100644
--- a/giraph-core/src/main/java/org/apache/giraph/zk/ZooKeeperExt.java
+++ b/giraph-core/src/main/java/org/apache/giraph/zk/ZooKeeperExt.java
@@ -162,8 +162,11 @@ public class ZooKeeperExt {
             if (progressable != null) {
               progressable.progress();
             }
-            zooKeeper.create(
-                path.substring(0, pos), null, acl, CreateMode.PERSISTENT);
+            String filePath = path.substring(0, pos);
+            if (zooKeeper.exists(filePath, false) == null) {
+              zooKeeper.create(
+                  filePath, null, acl, CreateMode.PERSISTENT);
+            }
           } catch (KeeperException.NodeExistsException e) {
             if (LOG.isDebugEnabled()) {
               LOG.debug("createExt: Znode " + path.substring(0, pos) +

Reply via email to