flink-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From u..@apache.org
Subject [1/2] flink git commit: [FLINK-3381] [runtime] Handle ConnectionLoss in ZooKeeper test util
Date Fri, 19 Feb 2016 23:14:01 GMT
Repository: flink
Updated Branches:
  refs/heads/master 8912fb06b -> a3b9d9717


[FLINK-3381] [runtime] Handle ConnectionLoss in ZooKeeper test util


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/59f978f8
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/59f978f8
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/59f978f8

Branch: refs/heads/master
Commit: 59f978f878ff30e1ca177d3fda11af803b59bb0b
Parents: 8912fb0
Author: Ufuk Celebi <uce@apache.org>
Authored: Fri Feb 19 11:24:33 2016 +0100
Committer: Ufuk Celebi <uce@apache.org>
Committed: Sat Feb 20 00:10:29 2016 +0100

----------------------------------------------------------------------
 .../runtime/zookeeper/ZooKeeperTestEnvironment.java | 16 +++++++++++-----
 1 file changed, 11 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/59f978f8/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java
b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java
index 5aca687..8fc80e0 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java
@@ -25,6 +25,7 @@ import org.apache.curator.utils.ZKPaths;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.zookeeper.KeeperException;
 
 import java.util.List;
 
@@ -137,17 +138,22 @@ public class ZooKeeperTestEnvironment {
 	 */
 	public void deleteAll() throws Exception {
 		final String path = "/" + client.getNamespace();
-		
-		for (int attempt = 0; attempt < 3; attempt++) {
+
+		int maxAttempts = 10;
+
+		for (int i = 0; i < maxAttempts; i++) {
 			try {
 				ZKPaths.deleteChildren(client.getZookeeperClient().getZooKeeper(), path, false);
 				break;
 			}
 			catch (org.apache.zookeeper.KeeperException.NoNodeException e) {
 				// that seems all right. if one of the children we want to delete is
-				// actually already deleted, that's fine. 
-				
-				// still, fall through the loop to delete the other children 
+				// actually already deleted, that's fine.
+				break;
+			}
+			catch (KeeperException.ConnectionLossException e) {
+				// Keep retrying
+				Thread.sleep(100);
 			}
 		}
 	}


Mime
View raw message