From 1fcec3747bf88d4d136cbe46e02c0f3b2677b37a Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Mon, 22 Aug 2016 10:27:26 -0400 Subject: [PATCH] NIFI-2617: Instead of retrying an infinite number of times to communicate with ZooKeeper, should try only for a short period in CuratorLeaderElectionManager. This is because web requests may be blocked waiting on a determination of which node is cluster coordinator Signed-off-by: Yolanda M. Davis This closes #906 --- .../election/CuratorLeaderElectionManager.java | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/leader/election/CuratorLeaderElectionManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/leader/election/CuratorLeaderElectionManager.java index 8c847713ad..977580e452 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/leader/election/CuratorLeaderElectionManager.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/leader/election/CuratorLeaderElectionManager.java @@ -28,7 +28,7 @@ import org.apache.curator.framework.recipes.leader.LeaderSelectorListener; import org.apache.curator.framework.recipes.leader.LeaderSelectorListenerAdapter; import org.apache.curator.framework.recipes.leader.Participant; import org.apache.curator.framework.state.ConnectionState; -import org.apache.curator.retry.RetryForever; +import org.apache.curator.retry.RetryNTimes; import org.apache.nifi.controller.cluster.ZooKeeperClientConfig; import org.apache.nifi.engine.FlowEngine; import org.apache.nifi.util.NiFiProperties; @@ -63,7 +63,7 @@ public class CuratorLeaderElectionManager implements LeaderElectionManager { stopped = false; - final RetryPolicy retryPolicy = new RetryForever(5000); + final RetryPolicy retryPolicy = new RetryNTimes(1, 100); curatorClient = CuratorFrameworkFactory.builder() .connectString(zkConfig.getConnectString()) .sessionTimeoutMs(zkConfig.getSessionTimeoutMillis()) @@ -177,9 +177,13 @@ public class CuratorLeaderElectionManager implements LeaderElectionManager { return "CuratorLeaderElectionManager[stopped=" + isStopped() + "]"; } + private synchronized LeaderRole getLeaderRole(final String roleName) { + return leaderRoles.get(roleName); + } + @Override - public synchronized boolean isLeader(final String roleName) { - final LeaderRole role = leaderRoles.get(roleName); + public boolean isLeader(final String roleName) { + final LeaderRole role = getLeaderRole(roleName); if (role == null) { return false; } @@ -188,8 +192,8 @@ public class CuratorLeaderElectionManager implements LeaderElectionManager { } @Override - public synchronized String getLeader(final String roleName) { - final LeaderRole role = leaderRoles.get(roleName); + public String getLeader(final String roleName) { + final LeaderRole role = getLeaderRole(roleName); if (role == null) { return null; }