From 90dc74627157296b4084ed65c1b752bfe25db1e8 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Sat, 13 Aug 2016 19:38:07 -0400 Subject: [PATCH 1/4] NIFI-2566: Refactored to allow just the Leader Election Manager to be responsible for determining who is the Cluster Coordinator --- .../heartbeat/HeartbeatMonitor.java | 5 + .../nifi-cluster-protocol-context.xml | 7 + .../ClusterProtocolHeartbeatMonitor.java | 104 +++++++++------ .../node/CuratorNodeProtocolSender.java | 124 ------------------ .../LeaderElectionNodeProtocolSender.java | 80 +++++++++++ .../node/NodeClusterCoordinator.java | 67 ++-------- .../NodeClusterCoordinatorFactoryBean.java | 4 +- .../nifi-cluster-manager-context.xml | 6 + .../TestAbstractHeartbeatMonitor.java | 5 + .../TestClusterProtocolHeartbeatMonitor.java | 67 ++++++++++ .../node/TestNodeClusterCoordinator.java | 12 +- .../nifi/cluster/integration/Cluster.java | 17 +++ .../integration/ClusterConnectionIT.java | 15 ++- .../cluster/integration/ClusterUtils.java | 10 +- .../apache/nifi/cluster/integration/Node.java | 28 ++-- .../nifi/controller/FlowController.java | 19 ++- .../cluster/ClusterProtocolHeartbeater.java | 82 ++---------- .../CuratorLeaderElectionManager.java | 66 +++++++++- .../election/LeaderElectionManager.java | 24 +++- .../spring/FlowControllerFactoryBean.java | 14 +- .../src/main/resources/nifi-context.xml | 1 + 21 files changed, 421 insertions(+), 336 deletions(-) delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/CuratorNodeProtocolSender.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/LeaderElectionNodeProtocolSender.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestClusterProtocolHeartbeatMonitor.java diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/HeartbeatMonitor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/HeartbeatMonitor.java index c151382319c5..988ba755e18c 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/HeartbeatMonitor.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/HeartbeatMonitor.java @@ -52,4 +52,9 @@ public interface HeartbeatMonitor { * @param nodeId the id of the node whose heartbeat should be removed */ void removeHeartbeat(NodeIdentifier nodeId); + + /** + * @return the address that heartbeats should be sent to when this node is elected coordinator. + */ + String getHeartbeatAddress(); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/resources/nifi-cluster-protocol-context.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/resources/nifi-cluster-protocol-context.xml index 04332a5d41d4..032081b2e95a 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/resources/nifi-cluster-protocol-context.xml +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/resources/nifi-cluster-protocol-context.xml @@ -54,11 +54,18 @@ + + + + + + diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/ClusterProtocolHeartbeatMonitor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/ClusterProtocolHeartbeatMonitor.java index d2d81d1da46e..5ecd8cbd17da 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/ClusterProtocolHeartbeatMonitor.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/ClusterProtocolHeartbeatMonitor.java @@ -33,6 +33,8 @@ import org.apache.curator.RetryPolicy; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.framework.state.ConnectionState; +import org.apache.curator.framework.state.ConnectionStateListener; import org.apache.curator.retry.RetryForever; import org.apache.nifi.cluster.HeartbeatPayload; import org.apache.nifi.cluster.coordination.ClusterCoordinator; @@ -57,7 +59,7 @@ * Uses Apache ZooKeeper to advertise the address to send heartbeats to, and then relies on the NiFi Cluster * Protocol to receive heartbeat messages from nodes in the cluster. */ -public class ClusterProtocolHeartbeatMonitor extends AbstractHeartbeatMonitor implements HeartbeatMonitor, ProtocolHandler { +public class ClusterProtocolHeartbeatMonitor extends AbstractHeartbeatMonitor implements HeartbeatMonitor, ProtocolHandler, ConnectionStateListener { protected static final Logger logger = LoggerFactory.getLogger(ClusterProtocolHeartbeatMonitor.class); private static final String COORDINATOR_ZNODE_NAME = "coordinator"; @@ -110,6 +112,11 @@ public ClusterProtocolHeartbeatMonitor(final ClusterCoordinator clusterCoordinat heartbeatAddress = hostname + ":" + port; } + @Override + public String getHeartbeatAddress() { + return heartbeatAddress; + } + @Override public void onStart() { final RetryPolicy retryPolicy = new RetryForever(5000); @@ -120,6 +127,7 @@ public void onStart() { .retryPolicy(retryPolicy) .defaultData(new byte[0]) .build(); + curatorClient.getConnectionStateListenable().addListener(this); curatorClient.start(); // We don't know what the heartbeats look like for the nodes, since we were just elected to monitoring @@ -133,48 +141,6 @@ public void onStart() { clusterCoordinator.getConnectionStatus(nodeId), Collections.emptySet(), 0, 0L, 0, System.currentTimeMillis()); heartbeatMessages.put(nodeId, heartbeat); } - - final Thread publishAddress = new Thread(new Runnable() { - @Override - public void run() { - while (!isStopped()) { - final String path = clusterNodesPath + "/" + COORDINATOR_ZNODE_NAME; - try { - try { - curatorClient.setData().forPath(path, heartbeatAddress.getBytes(StandardCharsets.UTF_8)); - logger.info("Successfully published Cluster Heartbeat Monitor Address of {} to ZooKeeper", heartbeatAddress); - return; - } catch (final NoNodeException nne) { - // ensure that parents are created, using a wide-open ACL because the parents contain no data - // and the path is not in any way sensitive. - try { - curatorClient.create().creatingParentsIfNeeded().withMode(CreateMode.EPHEMERAL).forPath(path); - } catch (final NodeExistsException nee) { - // This is okay. Node already exists. - } - - curatorClient.create().withMode(CreateMode.EPHEMERAL).forPath(path, heartbeatAddress.getBytes(StandardCharsets.UTF_8)); - logger.info("Successfully published address as heartbeat monitor address at path {} with value {}", path, heartbeatAddress); - - return; - } - } catch (Exception e) { - logger.warn("Failed to update ZooKeeper to notify nodes of the heartbeat address. Will continue to retry."); - - try { - Thread.sleep(2000L); - } catch (final InterruptedException ie) { - Thread.currentThread().interrupt(); - return; - } - } - } - } - }); - - publishAddress.setName("Publish Heartbeat Address"); - publishAddress.setDaemon(true); - publishAddress.start(); } @Override @@ -231,4 +197,56 @@ public ProtocolMessage handle(final ProtocolMessage msg) throws ProtocolExceptio public boolean canHandle(ProtocolMessage msg) { return msg.getType() == MessageType.HEARTBEAT; } + + protected void publishAddressAsync() { + final Thread publishAddressThread = new Thread(() -> publishAddress()); + + publishAddressThread.setName("Publish Heartbeat Address"); + publishAddressThread.setDaemon(true); + publishAddressThread.start(); + } + + protected void publishAddress() { + if (true) { + return; + } + while (!isStopped()) { + final String path = clusterNodesPath + "/" + COORDINATOR_ZNODE_NAME; + try { + try { + curatorClient.setData().forPath(path, heartbeatAddress.getBytes(StandardCharsets.UTF_8)); + logger.info("Successfully published Cluster Heartbeat Monitor Address of {} to ZooKeeper", heartbeatAddress); + return; + } catch (final NoNodeException nne) { + // ensure that parents are created, using a wide-open ACL because the parents contain no data + // and the path is not in any way sensitive. + try { + curatorClient.create().creatingParentsIfNeeded().withMode(CreateMode.EPHEMERAL).forPath(path, heartbeatAddress.getBytes(StandardCharsets.UTF_8)); + } catch (final NodeExistsException nee) { + // This is okay. Node already exists. + } + + logger.info("Successfully published address as heartbeat monitor address at path {} with value {}", path, heartbeatAddress); + + return; + } + } catch (Exception e) { + logger.warn("Failed to update ZooKeeper to notify nodes of the heartbeat address. Will continue to retry."); + + try { + Thread.sleep(2000L); + } catch (final InterruptedException ie) { + Thread.currentThread().interrupt(); + return; + } + } + } + } + + @Override + public void stateChanged(final CuratorFramework client, final ConnectionState newState) { + if (newState.isConnected()) { + publishAddressAsync(); + } + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/CuratorNodeProtocolSender.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/CuratorNodeProtocolSender.java deleted file mode 100644 index daa3e5c8121b..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/CuratorNodeProtocolSender.java +++ /dev/null @@ -1,124 +0,0 @@ -/* - * 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.nifi.cluster.coordination.node; - -import java.io.IOException; -import java.net.InetSocketAddress; -import java.nio.charset.StandardCharsets; -import java.util.Properties; - -import org.apache.curator.RetryPolicy; -import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.framework.CuratorFrameworkFactory; -import org.apache.curator.retry.RetryNTimes; -import org.apache.nifi.cluster.manager.exception.NoClusterCoordinatorException; -import org.apache.nifi.cluster.protocol.AbstractNodeProtocolSender; -import org.apache.nifi.cluster.protocol.ProtocolContext; -import org.apache.nifi.cluster.protocol.ProtocolException; -import org.apache.nifi.cluster.protocol.message.ProtocolMessage; -import org.apache.nifi.controller.cluster.ZooKeeperClientConfig; -import org.apache.nifi.io.socket.SocketConfiguration; -import org.apache.zookeeper.KeeperException.NoNodeException; -import org.apache.zookeeper.WatchedEvent; -import org.apache.zookeeper.Watcher; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Uses Apache Curator to determine the address of the current cluster coordinator - */ -public class CuratorNodeProtocolSender extends AbstractNodeProtocolSender { - private static final Logger logger = LoggerFactory.getLogger(CuratorNodeProtocolSender.class); - - private final String coordinatorPath; - private final ZooKeeperClientConfig zkConfig; - private InetSocketAddress coordinatorAddress; - - - public CuratorNodeProtocolSender(final SocketConfiguration socketConfig, final ProtocolContext protocolContext, final Properties properties) { - super(socketConfig, protocolContext); - zkConfig = ZooKeeperClientConfig.createConfig(properties); - coordinatorPath = zkConfig.resolvePath("cluster/nodes/coordinator"); - } - - @Override - protected synchronized InetSocketAddress getServiceAddress() throws IOException { - if (coordinatorAddress != null) { - return coordinatorAddress; - } - - final RetryPolicy retryPolicy = new RetryNTimes(0, 0); - final CuratorFramework curatorClient = CuratorFrameworkFactory.newClient(zkConfig.getConnectString(), - zkConfig.getSessionTimeoutMillis(), zkConfig.getConnectionTimeoutMillis(), retryPolicy); - curatorClient.start(); - - try { - // Get coordinator address and add watcher to change who we are heartbeating to if the value changes. - final byte[] coordinatorAddressBytes = curatorClient.getData().usingWatcher(new Watcher() { - @Override - public void process(final WatchedEvent event) { - coordinatorAddress = null; - } - }).forPath(coordinatorPath); - - if (coordinatorAddressBytes == null || coordinatorAddressBytes.length == 0) { - throw new NoClusterCoordinatorException("No node has yet been elected Cluster Coordinator. Cannot establish connection to cluster yet."); - } - - final String address = new String(coordinatorAddressBytes, StandardCharsets.UTF_8); - - final String[] splits = address.split(":"); - if (splits.length != 2) { - final String message = String.format("Attempted to determine Cluster Coordinator address. Zookeeper indicates " - + "that address is %s, but this is not in the expected format of :", address); - logger.error(message); - throw new ProtocolException(message); - } - - logger.info("Determined that Cluster Coordinator is located at {}; will use this address for sending heartbeat messages", address); - - final String hostname = splits[0]; - final int port; - try { - port = Integer.parseInt(splits[1]); - if (port < 1 || port > 65535) { - throw new NumberFormatException("Port must be in the range of 1 - 65535 but got " + port); - } - } catch (final NumberFormatException nfe) { - final String message = String.format("Attempted to determine Cluster Coordinator address. Zookeeper indicates " - + "that address is %s, but the port is not a valid port number", address); - logger.error(message); - throw new ProtocolException(message); - } - - final InetSocketAddress socketAddress = InetSocketAddress.createUnresolved(hostname, port); - coordinatorAddress = socketAddress; - return socketAddress; - } catch (final NoNodeException nne) { - logger.info("No node has yet been elected Cluster Coordinator. Cannot establish connection to cluster yet."); - throw new NoClusterCoordinatorException("No node has yet been elected Cluster Coordinator. Cannot establish connection to cluster yet."); - } catch (final NoClusterCoordinatorException ncce) { - throw ncce; - } catch (Exception e) { - throw new IOException("Unable to determine Cluster Coordinator from ZooKeeper", e); - } finally { - curatorClient.close(); - } - } - -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/LeaderElectionNodeProtocolSender.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/LeaderElectionNodeProtocolSender.java new file mode 100644 index 000000000000..03de329092d9 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/LeaderElectionNodeProtocolSender.java @@ -0,0 +1,80 @@ +/* + * 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.nifi.cluster.coordination.node; + +import java.io.IOException; +import java.net.InetSocketAddress; + +import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.cluster.manager.exception.NoClusterCoordinatorException; +import org.apache.nifi.cluster.protocol.AbstractNodeProtocolSender; +import org.apache.nifi.cluster.protocol.ProtocolContext; +import org.apache.nifi.cluster.protocol.ProtocolException; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage; +import org.apache.nifi.controller.leader.election.LeaderElectionManager; +import org.apache.nifi.io.socket.SocketConfiguration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class LeaderElectionNodeProtocolSender extends AbstractNodeProtocolSender { + private static final Logger logger = LoggerFactory.getLogger(LeaderElectionNodeProtocolSender.class); + + private final LeaderElectionManager electionManager; + + public LeaderElectionNodeProtocolSender(final SocketConfiguration socketConfiguration, final ProtocolContext protocolContext, final LeaderElectionManager electionManager) { + super(socketConfiguration, protocolContext); + this.electionManager = electionManager; + } + + @Override + protected InetSocketAddress getServiceAddress() throws IOException { + final String address = electionManager.getLeader(ClusterRoles.CLUSTER_COORDINATOR); + + if (StringUtils.isEmpty(address)) { + throw new NoClusterCoordinatorException("No node has yet been elected Cluster Coordinator. Cannot establish connection to cluster yet."); + } + + final String[] splits = address.split(":"); + if (splits.length != 2) { + final String message = String.format("Attempted to determine Cluster Coordinator address. Zookeeper indicates " + + "that address is %s, but this is not in the expected format of :", address); + logger.error(message); + throw new ProtocolException(message); + } + + logger.info("Determined that Cluster Coordinator is located at {}; will use this address for sending heartbeat messages", address); + + final String hostname = splits[0]; + final int port; + try { + port = Integer.parseInt(splits[1]); + if (port < 1 || port > 65535) { + throw new NumberFormatException("Port must be in the range of 1 - 65535 but got " + port); + } + } catch (final NumberFormatException nfe) { + final String message = String.format("Attempted to determine Cluster Coordinator address. Zookeeper indicates " + + "that address is %s, but the port is not a valid port number", address); + logger.error(message); + throw new ProtocolException(message); + } + + final InetSocketAddress socketAddress = InetSocketAddress.createUnresolved(hostname, port); + return socketAddress; + } + +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java index ac39dc58b07c..67e74614cf27 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java @@ -18,14 +18,12 @@ package org.apache.nifi.cluster.coordination.node; import java.io.IOException; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Properties; import java.util.Set; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; @@ -36,10 +34,6 @@ import org.apache.commons.collections4.queue.CircularFifoQueue; import org.apache.commons.lang3.StringUtils; -import org.apache.curator.RetryPolicy; -import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.framework.CuratorFrameworkFactory; -import org.apache.curator.retry.RetryNTimes; import org.apache.nifi.cluster.coordination.ClusterCoordinator; import org.apache.nifi.cluster.coordination.http.HttpResponseMerger; import org.apache.nifi.cluster.coordination.http.StandardHttpResponseMerger; @@ -68,14 +62,11 @@ import org.apache.nifi.cluster.protocol.message.ProtocolMessage; import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType; import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage; -import org.apache.nifi.controller.cluster.ZooKeeperClientConfig; +import org.apache.nifi.controller.leader.election.LeaderElectionManager; import org.apache.nifi.events.EventReporter; import org.apache.nifi.reporting.Severity; import org.apache.nifi.services.FlowService; import org.apache.nifi.web.revision.RevisionManager; -import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.WatchedEvent; -import org.apache.zookeeper.Watcher; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -92,37 +83,23 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl private final EventReporter eventReporter; private final ClusterNodeFirewall firewall; private final RevisionManager revisionManager; - - // Curator used to determine which node is coordinator - private final CuratorFramework curatorClient; - private final String nodesPathPrefix; - private final String coordinatorPath; + private final LeaderElectionManager leaderElectionManager; private volatile FlowService flowService; private volatile boolean connected; - private volatile String coordinatorAddress; private volatile boolean closed = false; private final ConcurrentMap nodeStatuses = new ConcurrentHashMap<>(); private final ConcurrentMap> nodeEvents = new ConcurrentHashMap<>(); - public NodeClusterCoordinator(final ClusterCoordinationProtocolSenderListener senderListener, final EventReporter eventReporter, - final ClusterNodeFirewall firewall, final RevisionManager revisionManager, final Properties nifiProperties) { + public NodeClusterCoordinator(final ClusterCoordinationProtocolSenderListener senderListener, final EventReporter eventReporter, final LeaderElectionManager leaderElectionManager, + final ClusterNodeFirewall firewall, final RevisionManager revisionManager) { this.senderListener = senderListener; this.flowService = null; this.eventReporter = eventReporter; this.firewall = firewall; this.revisionManager = revisionManager; - - final RetryPolicy retryPolicy = new RetryNTimes(10, 500); - final ZooKeeperClientConfig zkConfig = ZooKeeperClientConfig.createConfig(nifiProperties); - - curatorClient = CuratorFrameworkFactory.newClient(zkConfig.getConnectString(), - zkConfig.getSessionTimeoutMillis(), zkConfig.getConnectionTimeoutMillis(), retryPolicy); - - curatorClient.start(); - nodesPathPrefix = zkConfig.resolvePath("cluster/nodes"); - coordinatorPath = nodesPathPrefix + "/coordinator"; + this.leaderElectionManager = leaderElectionManager; senderListener.addHandler(this); } @@ -138,10 +115,9 @@ public void shutdown() { final NodeConnectionStatus shutdownStatus = new NodeConnectionStatus(getLocalNodeIdentifier(), DisconnectionCode.NODE_SHUTDOWN); updateNodeStatus(shutdownStatus, false); logger.info("Successfully notified other nodes that I am shutting down"); - - curatorClient.close(); } + @Override public void setLocalNodeIdentifier(final NodeIdentifier nodeId) { this.nodeId = nodeId; @@ -174,6 +150,7 @@ private NodeIdentifier waitForNodeIdentifier(final Supplier fetc Thread.sleep(100L); } catch (final InterruptedException ie) { Thread.currentThread().interrupt(); + return null; } } } @@ -182,34 +159,12 @@ private NodeIdentifier waitForNodeIdentifier(final Supplier fetc } private String getElectedActiveCoordinatorAddress() throws IOException { - final String curAddress = coordinatorAddress; - if (curAddress != null) { - return curAddress; - } - - try { - // Get coordinator address and add watcher to change who we are heartbeating to if the value changes. - final byte[] coordinatorAddressBytes = curatorClient.getData().usingWatcher(new Watcher() { - @Override - public void process(final WatchedEvent event) { - coordinatorAddress = null; - } - }).forPath(coordinatorPath); - final String address = coordinatorAddress = new String(coordinatorAddressBytes, StandardCharsets.UTF_8); - - logger.info("Determined that Cluster Coordinator is located at {}", address); - return address; - } catch (final KeeperException.NoNodeException nne) { - throw new NoClusterCoordinatorException(); - } catch (Exception e) { - throw new IOException("Unable to determine Cluster Coordinator from ZooKeeper", e); - } + return leaderElectionManager.getLeader(ClusterRoles.CLUSTER_COORDINATOR); } @Override public void resetNodeStatuses(final Map statusMap) { logger.info("Resetting cluster node statuses from {} to {}", nodeStatuses, statusMap); - coordinatorAddress = null; // For each proposed replacement, update the nodeStatuses map if and only if the replacement // has a larger update id than the current value. @@ -558,6 +513,11 @@ private NodeIdentifier getElectedActiveCoordinatorNode(final boolean warnOnError return null; } + if (electedNodeAddress == null) { + logger.debug("There is currently no elected active Cluster Coordinator"); + return null; + } + final int colonLoc = electedNodeAddress.indexOf(':'); if (colonLoc < 1) { if (warnOnError) { @@ -1103,7 +1063,6 @@ private boolean isMissingCounter(final Set problematicNodeResponse @Override public void setConnected(final boolean connected) { this.connected = connected; - this.coordinatorAddress = null; // if connection state changed, we are not sure about the coordinator. Check for address again. } @Override diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/NodeClusterCoordinatorFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/NodeClusterCoordinatorFactoryBean.java index b414e0d58ae5..0b83c2343bcd 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/NodeClusterCoordinatorFactoryBean.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/NodeClusterCoordinatorFactoryBean.java @@ -20,6 +20,7 @@ import org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator; import org.apache.nifi.cluster.firewall.ClusterNodeFirewall; import org.apache.nifi.cluster.protocol.impl.ClusterCoordinationProtocolSenderListener; +import org.apache.nifi.controller.leader.election.LeaderElectionManager; import org.apache.nifi.events.EventReporter; import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.web.revision.RevisionManager; @@ -42,8 +43,9 @@ public NodeClusterCoordinator getObject() throws Exception { final EventReporter eventReporter = applicationContext.getBean("eventReporter", EventReporter.class); final ClusterNodeFirewall clusterFirewall = applicationContext.getBean("clusterFirewall", ClusterNodeFirewall.class); final RevisionManager revisionManager = applicationContext.getBean("revisionManager", RevisionManager.class); + final LeaderElectionManager electionManager = applicationContext.getBean("leaderElectionManager", LeaderElectionManager.class); - nodeClusterCoordinator = new NodeClusterCoordinator(protocolSenderListener, eventReporter, clusterFirewall, revisionManager, properties); + nodeClusterCoordinator = new NodeClusterCoordinator(protocolSenderListener, eventReporter, electionManager, clusterFirewall, revisionManager); } return nodeClusterCoordinator; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/resources/nifi-cluster-manager-context.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/resources/nifi-cluster-manager-context.xml index a95db4bc4670..38b3a5be18f6 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/resources/nifi-cluster-manager-context.xml +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/resources/nifi-cluster-manager-context.xml @@ -35,6 +35,12 @@ + + + + + + diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java index 5086dc004dfc..bb910a5bb6c1 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java @@ -360,5 +360,10 @@ void waitForProcessed() throws InterruptedException { mutex.wait(); } } + + @Override + public String getHeartbeatAddress() { + return "localhost"; + } } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestClusterProtocolHeartbeatMonitor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestClusterProtocolHeartbeatMonitor.java new file mode 100644 index 000000000000..1d95439c7c84 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestClusterProtocolHeartbeatMonitor.java @@ -0,0 +1,67 @@ +/* + * 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.nifi.cluster.coordination.heartbeat; + +import static org.junit.Assert.assertEquals; + +import java.util.Properties; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.curator.framework.state.ConnectionState; +import org.apache.nifi.cluster.coordination.ClusterCoordinator; +import org.apache.nifi.cluster.protocol.ProtocolListener; +import org.apache.nifi.util.NiFiProperties; +import org.junit.Test; +import org.mockito.Mockito; + +public class TestClusterProtocolHeartbeatMonitor { + + @Test + public void testRepublishesAddressOnZooKeeperReconnect() throws InterruptedException { + final ClusterCoordinator coordinator = Mockito.mock(ClusterCoordinator.class); + final ProtocolListener protocolListener = Mockito.mock(ProtocolListener.class); + final Properties properties = new Properties(); + properties.setProperty(NiFiProperties.CLUSTER_NODE_PROTOCOL_PORT, "0"); + properties.setProperty(NiFiProperties.ZOOKEEPER_CONNECT_STRING, "localhost:2181"); + + final AtomicInteger publishCount = new AtomicInteger(0); + final ClusterProtocolHeartbeatMonitor monitor = new ClusterProtocolHeartbeatMonitor(coordinator, protocolListener, properties) { + @Override + protected void publishAddress() { + publishCount.incrementAndGet(); + } + }; + + monitor.start(); + + Thread.sleep(250L); + assertEquals(0, publishCount.get()); + monitor.stateChanged(null, ConnectionState.CONNECTED); + + Thread.sleep(250L); + assertEquals(1, publishCount.get()); + + monitor.stateChanged(null, ConnectionState.LOST); + Thread.sleep(250L); + assertEquals(1, publishCount.get()); + + monitor.stateChanged(null, ConnectionState.CONNECTED); + Thread.sleep(250L); + assertEquals(2, publishCount.get()); + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/node/TestNodeClusterCoordinator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/node/TestNodeClusterCoordinator.java index 2f034b34b81b..9f6928942d01 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/node/TestNodeClusterCoordinator.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/node/TestNodeClusterCoordinator.java @@ -29,7 +29,6 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.Properties; import java.util.Set; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; @@ -60,11 +59,6 @@ public class TestNodeClusterCoordinator { private ClusterCoordinationProtocolSenderListener senderListener; private List nodeStatuses; - private Properties createProperties() { - final Properties props = new Properties(); - props.put("nifi.zookeeper.connect.string", "localhost:2181"); - return props; - } @Before public void setup() throws IOException { @@ -75,7 +69,7 @@ public void setup() throws IOException { final RevisionManager revisionManager = Mockito.mock(RevisionManager.class); Mockito.when(revisionManager.getAllRevisions()).thenReturn(Collections.emptyList()); - coordinator = new NodeClusterCoordinator(senderListener, eventReporter, null, revisionManager, createProperties()) { + coordinator = new NodeClusterCoordinator(senderListener, eventReporter, null, null, revisionManager) { @Override void notifyOthersOfNodeStatusChange(NodeConnectionStatus updatedStatus, boolean notifyAllNodes, boolean waitForCoordinator) { nodeStatuses.add(updatedStatus); @@ -130,7 +124,7 @@ public void testTryAgainIfNoFlowServiceSet() { final RevisionManager revisionManager = Mockito.mock(RevisionManager.class); Mockito.when(revisionManager.getAllRevisions()).thenReturn(Collections.emptyList()); - final NodeClusterCoordinator coordinator = new NodeClusterCoordinator(senderListener, eventReporter, null, revisionManager, createProperties()) { + final NodeClusterCoordinator coordinator = new NodeClusterCoordinator(senderListener, eventReporter, null, null, revisionManager) { @Override void notifyOthersOfNodeStatusChange(NodeConnectionStatus updatedStatus, boolean notifyAllNodes, boolean waitForCoordinator) { } @@ -168,7 +162,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { final RevisionManager revisionManager = Mockito.mock(RevisionManager.class); Mockito.when(revisionManager.getAllRevisions()).thenReturn(Collections.emptyList()); - final NodeClusterCoordinator coordinator = new NodeClusterCoordinator(senderListener, eventReporter, null, revisionManager, createProperties()) { + final NodeClusterCoordinator coordinator = new NodeClusterCoordinator(senderListener, eventReporter, null, null, revisionManager) { @Override void notifyOthersOfNodeStatusChange(NodeConnectionStatus updatedStatus, boolean notifyAllNodes, boolean waitForCoordinator) { } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java index dbd8c004113b..59a13b28fcd2 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java @@ -22,6 +22,10 @@ import java.util.Set; import java.util.concurrent.TimeUnit; +import org.apache.curator.RetryPolicy; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.retry.RetryNTimes; import org.apache.curator.test.TestingServer; import org.apache.nifi.cluster.coordination.node.ClusterRoles; import org.apache.nifi.util.NiFiProperties; @@ -89,6 +93,19 @@ public Set getNodes() { return Collections.unmodifiableSet(nodes); } + public CuratorFramework createCuratorClient() { + final RetryPolicy retryPolicy = new RetryNTimes(20, 500); + final CuratorFramework curatorClient = CuratorFrameworkFactory.builder() + .connectString(getZooKeeperConnectString()) + .sessionTimeoutMs(3000) + .connectionTimeoutMs(3000) + .retryPolicy(retryPolicy) + .defaultData(new byte[0]) + .build(); + + curatorClient.start(); + return curatorClient; + } public Node createNode() { NiFiProperties.getInstance().setProperty(NiFiProperties.ZOOKEEPER_CONNECT_STRING, getZooKeeperConnectString()); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/ClusterConnectionIT.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/ClusterConnectionIT.java index 6881ca233c1d..699d692cfef8 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/ClusterConnectionIT.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/ClusterConnectionIT.java @@ -125,7 +125,7 @@ public void testReconnectGetsCorrectClusterTopology() throws IOException { // wait for node 1 and 3 to recognize that node 2 is gone Stream.of(firstNode, thirdNode).forEach(node -> { - node.assertNodeDisconnects(secondNode.getIdentifier(), 5, TimeUnit.SECONDS); + node.assertNodeDisconnects(secondNode.getIdentifier(), 10, TimeUnit.SECONDS); }); // restart node @@ -222,17 +222,20 @@ public void testHeartbeatsMonitored() throws IOException { firstNode.waitUntilConnected(10, TimeUnit.SECONDS); secondNode.waitUntilConnected(10, TimeUnit.SECONDS); - secondNode.suspendHeartbeating(); + final Node nodeToSuspend = firstNode; + final Node otherNode = secondNode; + + System.out.println("\n\n\nSuspending heartbeats on node. Roles = " + nodeToSuspend.getRoles() + "\n\n\n"); + nodeToSuspend.suspendHeartbeating(); // Heartbeat interval in nifi.properties is set to 1 sec. This means that the node should be kicked out // due to lack of heartbeat after 8 times this amount of time, or 8 seconds. - firstNode.assertNodeDisconnects(secondNode.getIdentifier(), 12, TimeUnit.SECONDS); + otherNode.assertNodeDisconnects(nodeToSuspend.getIdentifier(), 12, TimeUnit.SECONDS); - secondNode.resumeHeartbeating(); - firstNode.assertNodeConnects(secondNode.getIdentifier(), 10, TimeUnit.SECONDS); + nodeToSuspend.resumeHeartbeating(); + otherNode.assertNodeConnects(nodeToSuspend.getIdentifier(), 10, TimeUnit.SECONDS); } finally { cluster.stop(); } } - } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/ClusterUtils.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/ClusterUtils.java index 972d2c7a5a5b..98d5cb334859 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/ClusterUtils.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/ClusterUtils.java @@ -24,13 +24,21 @@ public class ClusterUtils { public static void waitUntilConditionMet(final long time, final TimeUnit timeUnit, final BooleanSupplier test) { + waitUntilConditionMet(time, timeUnit, test, null); + } + + public static void waitUntilConditionMet(final long time, final TimeUnit timeUnit, final BooleanSupplier test, final Supplier errorMessageSupplier) { final long nanosToWait = timeUnit.toNanos(time); final long start = System.nanoTime(); final long maxTime = start + nanosToWait; while (!test.getAsBoolean()) { if (System.nanoTime() > maxTime) { - throw new AssertionError("Condition never occurred after waiting " + time + " " + timeUnit); + if (errorMessageSupplier == null) { + throw new AssertionError("Condition never occurred after waiting " + time + " " + timeUnit); + } else { + throw new AssertionError("Condition never occurred after waiting " + time + " " + timeUnit + " : " + errorMessageSupplier.get()); + } } } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Node.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Node.java index 5bfe83c6383b..ec57992b3aa7 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Node.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Node.java @@ -30,15 +30,15 @@ import org.apache.commons.lang3.builder.HashCodeBuilder; import org.apache.nifi.authorization.Authorizer; import org.apache.nifi.cluster.ReportedEvent; -import org.apache.nifi.cluster.coordination.ClusterCoordinator; import org.apache.nifi.cluster.coordination.heartbeat.ClusterProtocolHeartbeatMonitor; import org.apache.nifi.cluster.coordination.heartbeat.HeartbeatMonitor; -import org.apache.nifi.cluster.coordination.node.CuratorNodeProtocolSender; +import org.apache.nifi.cluster.coordination.node.LeaderElectionNodeProtocolSender; import org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator; import org.apache.nifi.cluster.coordination.node.NodeConnectionState; import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; import org.apache.nifi.cluster.protocol.ClusterCoordinationProtocolSender; import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.cluster.protocol.NodeProtocolSender; import org.apache.nifi.cluster.protocol.ProtocolContext; import org.apache.nifi.cluster.protocol.ProtocolListener; import org.apache.nifi.cluster.protocol.impl.ClusterCoordinationProtocolSenderListener; @@ -51,6 +51,8 @@ import org.apache.nifi.components.state.Scope; import org.apache.nifi.controller.FlowController; import org.apache.nifi.controller.StandardFlowService; +import org.apache.nifi.controller.leader.election.CuratorLeaderElectionManager; +import org.apache.nifi.controller.leader.election.LeaderElectionManager; import org.apache.nifi.controller.repository.FlowFileEventRepository; import org.apache.nifi.encrypt.StringEncryptor; import org.apache.nifi.engine.FlowEngine; @@ -74,9 +76,10 @@ public class Node { private final RevisionManager revisionManager; private NodeClusterCoordinator clusterCoordinator; - private CuratorNodeProtocolSender protocolSender; + private NodeProtocolSender protocolSender; private FlowController flowController; private StandardFlowService flowService; + private LeaderElectionManager electionManager; private ProtocolListener protocolListener; @@ -97,6 +100,8 @@ public Node(final NodeIdentifier nodeId, final NiFiProperties properties) { revisionManager = Mockito.mock(RevisionManager.class); Mockito.when(revisionManager.getAllRevisions()).thenReturn(Collections. emptyList()); + + electionManager = new CuratorLeaderElectionManager(4, nodeProperties); } @@ -110,7 +115,8 @@ public synchronized void start() { final HeartbeatMonitor heartbeatMonitor = createHeartbeatMonitor(); flowController = FlowController.createClusteredInstance(Mockito.mock(FlowFileEventRepository.class), nodeProperties, - null, null, StringEncryptor.createEncryptor(), protocolSender, Mockito.mock(BulletinRepository.class), clusterCoordinator, heartbeatMonitor, VariableRegistry.EMPTY_REGISTRY); + null, null, StringEncryptor.createEncryptor(), protocolSender, Mockito.mock(BulletinRepository.class), clusterCoordinator, + heartbeatMonitor, electionManager, VariableRegistry.EMPTY_REGISTRY); try { flowController.initializeFlow(); @@ -205,13 +211,13 @@ public NodeConnectionStatus getConnectionStatus() { } @SuppressWarnings("unchecked") - private CuratorNodeProtocolSender createNodeProtocolSender() { + private NodeProtocolSender createNodeProtocolSender() { final SocketConfiguration socketConfig = new SocketConfiguration(); socketConfig.setSocketTimeout(3000); socketConfig.setReuseAddress(true); final ProtocolContext protocolContext = new JaxbProtocolContext<>(JaxbProtocolUtils.JAXB_CONTEXT); - final CuratorNodeProtocolSender protocolSender = new CuratorNodeProtocolSender(socketConfig, protocolContext, nodeProperties); + final NodeProtocolSender protocolSender = new LeaderElectionNodeProtocolSender(socketConfig, protocolContext, electionManager); return protocolSender; } @@ -250,11 +256,11 @@ public void reportEvent(Severity severity, String category, String message) { } final ClusterCoordinationProtocolSenderListener protocolSenderListener = new ClusterCoordinationProtocolSenderListener(createCoordinatorProtocolSender(), protocolListener); - return new NodeClusterCoordinator(protocolSenderListener, eventReporter, null, revisionManager, nodeProperties); + return new NodeClusterCoordinator(protocolSenderListener, eventReporter, electionManager, null, revisionManager); } - public ClusterCoordinator getClusterCoordinator() { + public NodeClusterCoordinator getClusterCoordinator() { return clusterCoordinator; } @@ -292,7 +298,8 @@ public void waitUntilElectedForRole(final String roleName, final long time, fina */ public void assertNodeConnects(final NodeIdentifier nodeId, final long time, final TimeUnit timeUnit) { ClusterUtils.waitUntilConditionMet(time, timeUnit, - () -> getClusterCoordinator().getConnectionStatus(nodeId).getState() == NodeConnectionState.CONNECTED); + () -> getClusterCoordinator().getConnectionStatus(nodeId).getState() == NodeConnectionState.CONNECTED, + () -> "Connection Status is " + getClusterCoordinator().getConnectionStatus(nodeId).toString()); } @@ -305,7 +312,8 @@ public void assertNodeConnects(final NodeIdentifier nodeId, final long time, fin */ public void assertNodeDisconnects(final NodeIdentifier nodeId, final long time, final TimeUnit timeUnit) { ClusterUtils.waitUntilConditionMet(time, timeUnit, - () -> getClusterCoordinator().getConnectionStatus(nodeId).getState() == NodeConnectionState.DISCONNECTED); + () -> getClusterCoordinator().getConnectionStatus(nodeId).getState() == NodeConnectionState.DISCONNECTED, + () -> "Connection Status is " + getClusterCoordinator().getConnectionStatus(nodeId).toString()); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java index 2635fc472b14..6599bccdfabb 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java @@ -96,7 +96,6 @@ import org.apache.nifi.controller.exception.ProcessorInstantiationException; import org.apache.nifi.controller.label.Label; import org.apache.nifi.controller.label.StandardLabel; -import org.apache.nifi.controller.leader.election.CuratorLeaderElectionManager; import org.apache.nifi.controller.leader.election.LeaderElectionManager; import org.apache.nifi.controller.leader.election.LeaderElectionStateChangeListener; import org.apache.nifi.controller.queue.FlowFileQueue; @@ -389,6 +388,7 @@ public static FlowController createStandaloneInstance( bulletinRepo, /* cluster coordinator */ null, /* heartbeat monitor */ null, + /* leader election manager */ null, /* variable registry */ variableRegistry); } @@ -402,7 +402,8 @@ public static FlowController createClusteredInstance( final BulletinRepository bulletinRepo, final ClusterCoordinator clusterCoordinator, final HeartbeatMonitor heartbeatMonitor, - VariableRegistry variableRegistry) { + final LeaderElectionManager leaderElectionManager, + final VariableRegistry variableRegistry) { final FlowController flowController = new FlowController( flowFileEventRepo, @@ -414,7 +415,9 @@ public static FlowController createClusteredInstance( protocolSender, bulletinRepo, clusterCoordinator, - heartbeatMonitor, variableRegistry); + heartbeatMonitor, + leaderElectionManager, + variableRegistry); return flowController; } @@ -430,6 +433,7 @@ private FlowController( final BulletinRepository bulletinRepo, final ClusterCoordinator clusterCoordinator, final HeartbeatMonitor heartbeatMonitor, + final LeaderElectionManager leaderElectionManager, final VariableRegistry variableRegistry) { maxTimerDrivenThreads = new AtomicInteger(10); @@ -579,10 +583,10 @@ public void run() { this.connectionStatus = new NodeConnectionStatus(nodeId, DisconnectionCode.NOT_YET_CONNECTED); heartbeatBeanRef.set(new HeartbeatBean(rootGroup, false)); + this.leaderElectionManager = leaderElectionManager; if (configuredForClustering) { - leaderElectionManager = new CuratorLeaderElectionManager(4, properties); - heartbeater = new ClusterProtocolHeartbeater(protocolSender, properties); + heartbeater = new ClusterProtocolHeartbeater(protocolSender, leaderElectionManager); // Check if there is already a cluster coordinator elected. If not, go ahead // and register for coordinator role. If there is already one elected, do not register until @@ -602,7 +606,6 @@ public void run() { leaderElectionManager.start(); } else { - leaderElectionManager = null; heartbeater = null; } } @@ -3308,6 +3311,8 @@ public boolean isConfiguredForClustering() { private void registerForClusterCoordinator() { + final String participantId = heartbeatMonitor.getHeartbeatAddress(); + leaderElectionManager.register(ClusterRoles.CLUSTER_COORDINATOR, new LeaderElectionStateChangeListener() { @Override public synchronized void onLeaderRelinquish() { @@ -3336,7 +3341,7 @@ public synchronized void onLeaderElection() { clusterCoordinator.addRole(ClusterRoles.CLUSTER_COORDINATOR); } } - }); + }, participantId); } private void registerForPrimaryNode() { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/cluster/ClusterProtocolHeartbeater.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/cluster/ClusterProtocolHeartbeater.java index 02403184ec36..eb3cebe73b53 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/cluster/ClusterProtocolHeartbeater.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/cluster/ClusterProtocolHeartbeater.java @@ -18,102 +18,44 @@ package org.apache.nifi.controller.cluster; import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.util.Properties; -import org.apache.curator.RetryPolicy; -import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.framework.CuratorFrameworkFactory; -import org.apache.curator.retry.RetryNTimes; +import org.apache.nifi.cluster.coordination.node.ClusterRoles; import org.apache.nifi.cluster.protocol.NodeProtocolSender; import org.apache.nifi.cluster.protocol.ProtocolException; import org.apache.nifi.cluster.protocol.message.HeartbeatMessage; -import org.apache.zookeeper.WatchedEvent; -import org.apache.zookeeper.Watcher; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import org.apache.nifi.controller.leader.election.LeaderElectionManager; /** - * Uses ZooKeeper in order to determine which node is the elected Cluster Coordinator and to indicate - * that this node is part of the cluster. However, once the Cluster Coordinator is known, heartbeats are + * Uses Leader Election Manager in order to determine which node is the elected Cluster Coordinator and to indicate + * that this node is part of the cluster. Once the Cluster Coordinator is known, heartbeats are * sent directly to the Cluster Coordinator. */ public class ClusterProtocolHeartbeater implements Heartbeater { - private static final Logger logger = LoggerFactory.getLogger(ClusterProtocolHeartbeater.class); - private final NodeProtocolSender protocolSender; - private final CuratorFramework curatorClient; - private final String nodesPathPrefix; - - private final String coordinatorPath; - private volatile String coordinatorAddress; + private final LeaderElectionManager electionManager; - - public ClusterProtocolHeartbeater(final NodeProtocolSender protocolSender, final Properties properties) { + public ClusterProtocolHeartbeater(final NodeProtocolSender protocolSender, final LeaderElectionManager electionManager) { this.protocolSender = protocolSender; - - final RetryPolicy retryPolicy = new RetryNTimes(10, 500); - final ZooKeeperClientConfig zkConfig = ZooKeeperClientConfig.createConfig(properties); - - curatorClient = CuratorFrameworkFactory.newClient(zkConfig.getConnectString(), - zkConfig.getSessionTimeoutMillis(), zkConfig.getConnectionTimeoutMillis(), retryPolicy); - - curatorClient.start(); - nodesPathPrefix = zkConfig.resolvePath("cluster/nodes"); - coordinatorPath = nodesPathPrefix + "/coordinator"; + this.electionManager = electionManager; } @Override public String getHeartbeatAddress() throws IOException { - final String curAddress = coordinatorAddress; - if (curAddress != null) { - return curAddress; + final String heartbeatAddress = electionManager.getLeader(ClusterRoles.CLUSTER_COORDINATOR); + if (heartbeatAddress == null) { + throw new ProtocolException("Cannot send heartbeat because there is no Cluster Coordinator currently elected"); } - try { - // Get coordinator address and add watcher to change who we are heartbeating to if the value changes. - final byte[] coordinatorAddressBytes = curatorClient.getData().usingWatcher(new Watcher() { - @Override - public void process(final WatchedEvent event) { - coordinatorAddress = null; - } - }).forPath(coordinatorPath); - final String address = coordinatorAddress = new String(coordinatorAddressBytes, StandardCharsets.UTF_8); - - logger.info("Determined that Cluster Coordinator is located at {}; will use this address for sending heartbeat messages", address); - return address; - } catch (Exception e) { - throw new IOException("Unable to determine Cluster Coordinator from ZooKeeper", e); - } + return heartbeatAddress; } - @Override public synchronized void send(final HeartbeatMessage heartbeatMessage) throws IOException { final String heartbeatAddress = getHeartbeatAddress(); - - try { - protocolSender.heartbeat(heartbeatMessage, heartbeatAddress); - } catch (final ProtocolException pe) { - // a ProtocolException is likely the result of not being able to communicate - // with the coordinator. If we do get an IOException communicating with the coordinator, - // it will be the cause of the Protocol Exception. In this case, set coordinatorAddress - // to null so that we double-check next time that the coordinator has not changed. - if (pe.getCause() instanceof IOException) { - coordinatorAddress = null; - } - - throw pe; - } + protocolSender.heartbeat(heartbeatMessage, heartbeatAddress); } - @Override public void close() throws IOException { - if (curatorClient != null) { - curatorClient.close(); - } - - logger.info("ZooKeeper heartbeater closed. Will no longer send Heartbeat messages to ZooKeeper"); } } 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 7bf749410d76..14351828364d 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 @@ -21,12 +21,14 @@ import java.util.Map; import java.util.Properties; +import org.apache.commons.lang3.StringUtils; import org.apache.curator.RetryPolicy; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.framework.recipes.leader.LeaderSelector; 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.nifi.controller.cluster.ZooKeeperClientConfig; @@ -47,7 +49,7 @@ public class CuratorLeaderElectionManager implements LeaderElectionManager { private volatile boolean stopped = true; private final Map leaderRoles = new HashMap<>(); - private final Map registeredRoles = new HashMap<>(); + private final Map registeredRoles = new HashMap<>(); public CuratorLeaderElectionManager(final int threadPoolSize) { @@ -82,8 +84,9 @@ public synchronized void start() { // Call #register for each already-registered role. This will // cause us to start listening for leader elections for that // role again - for (final Map.Entry entry : registeredRoles.entrySet()) { - register(entry.getKey(), entry.getValue()); + for (final Map.Entry entry : registeredRoles.entrySet()) { + final RegisteredRole role = entry.getValue(); + register(entry.getKey(), role.getListener(), role.getParticipantId()); } logger.info("{} started", this); @@ -97,7 +100,12 @@ public synchronized void register(final String roleName) { @Override - public synchronized void register(final String roleName, final LeaderElectionStateChangeListener listener) { + public void register(String roleName, LeaderElectionStateChangeListener listener) { + register(roleName, listener, null); + } + + @Override + public synchronized void register(final String roleName, final LeaderElectionStateChangeListener listener, final String participantId) { logger.debug("{} Registering new Leader Selector for role {}", this, roleName); if (leaderRoles.containsKey(roleName)) { @@ -114,18 +122,23 @@ public synchronized void register(final String roleName, final LeaderElectionSta throw new IllegalStateException("Cannot register leader election for role '" + roleName + "' because this is not a valid role name"); } - registeredRoles.put(roleName, listener); + registeredRoles.put(roleName, new RegisteredRole(participantId, listener)); if (!isStopped()) { final ElectionListener electionListener = new ElectionListener(roleName, listener); final LeaderSelector leaderSelector = new LeaderSelector(curatorClient, leaderPath, leaderElectionMonitorEngine, electionListener); leaderSelector.autoRequeue(); + if (participantId != null) { + leaderSelector.setId(participantId); + } + leaderSelector.start(); final LeaderRole leaderRole = new LeaderRole(leaderSelector, electionListener); leaderRoles.put(roleName, leaderRole); } + logger.info("{} Registered new Leader Selector for role {}", this, roleName); } @@ -185,6 +198,32 @@ public synchronized boolean isLeader(final String roleName) { return role.isLeader(); } + @Override + public synchronized String getLeader(final String roleName) { + final LeaderRole role = leaderRoles.get(roleName); + if (role == null) { + return null; + } + + Participant participant; + try { + participant = role.getLeaderSelector().getLeader(); + } catch (Exception e) { + logger.debug("Unable to determine leader for role '{}'; returning null", roleName); + return null; + } + + if (participant == null) { + return null; + } + + final String participantId = participant.getId(); + if (StringUtils.isEmpty(participantId)) { + return null; + } + + return participantId; + } private static class LeaderRole { private final LeaderSelector leaderSelector; @@ -204,6 +243,23 @@ public boolean isLeader() { } } + private static class RegisteredRole { + private final LeaderElectionStateChangeListener listener; + private final String participantId; + + public RegisteredRole(final String participantId, final LeaderElectionStateChangeListener listener) { + this.participantId = participantId; + this.listener = listener; + } + + public LeaderElectionStateChangeListener getListener() { + return listener; + } + + public String getParticipantId() { + return participantId; + } + } private class ElectionListener extends LeaderSelectorListenerAdapter implements LeaderSelectorListener { private final String roleName; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/leader/election/LeaderElectionManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/leader/election/LeaderElectionManager.java index d16dbdbebd86..ef36528c9da6 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/leader/election/LeaderElectionManager.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/leader/election/LeaderElectionManager.java @@ -31,7 +31,7 @@ public interface LeaderElectionManager { void register(String roleName); /** - * Adds a new role for which a leader is required + * Adds a new role for which a leader is required, without providing a Participant ID * * @param roleName the name of the role * @param listener a listener that will be called when the node gains or relinquishes @@ -39,6 +39,28 @@ public interface LeaderElectionManager { */ void register(String roleName, LeaderElectionStateChangeListener listener); + /** + * Adds a new role for which a leader is required, providing the given value for this node as the Participant ID + * + * @param roleName the name of the role + * @param listener a listener that will be called when the node gains or relinquishes + * the role of leader + * @param participantId the ID to register as this node's Participant ID. All nodes will see this as the identifier when + * asking to see who the leader is via the {@link #getLeader(String)} method + */ + void register(String roleName, LeaderElectionStateChangeListener listener, String participantId); + + /** + * Returns the Participant ID of the node that is elected the leader, if one was provided when the node registered + * for the role via {@link #register(String, LeaderElectionStateChangeListener, String)}. If there is currently no leader + * known or if the role was registered without providing a Participant ID, this will return null. + * + * @param roleName the name of the role + * @return the Participant ID of the node that is elected leader, or null if either no leader is known or the leader + * did not register with a Participant ID. + */ + String getLeader(String roleName); + /** * Removes the role with the given name from this manager. If this * node is the elected leader for the given role, this node will relinquish diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/FlowControllerFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/FlowControllerFactoryBean.java index 2760ca94c8c9..7ed91876258f 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/FlowControllerFactoryBean.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/FlowControllerFactoryBean.java @@ -22,13 +22,12 @@ import org.apache.nifi.cluster.coordination.heartbeat.HeartbeatMonitor; import org.apache.nifi.cluster.protocol.NodeProtocolSender; import org.apache.nifi.controller.FlowController; +import org.apache.nifi.controller.leader.election.LeaderElectionManager; import org.apache.nifi.controller.repository.FlowFileEventRepository; import org.apache.nifi.encrypt.StringEncryptor; import org.apache.nifi.registry.VariableRegistry; import org.apache.nifi.reporting.BulletinRepository; import org.apache.nifi.util.NiFiProperties; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.springframework.beans.BeansException; import org.springframework.beans.factory.FactoryBean; import org.springframework.context.ApplicationContext; @@ -40,8 +39,6 @@ @SuppressWarnings("rawtypes") public class FlowControllerFactoryBean implements FactoryBean, ApplicationContextAware { - private static final Logger LOG = LoggerFactory.getLogger(FlowControllerFactoryBean.class); - private ApplicationContext applicationContext; private FlowController flowController; private NiFiProperties properties; @@ -51,6 +48,7 @@ public class FlowControllerFactoryBean implements FactoryBean, ApplicationContex private BulletinRepository bulletinRepository; private ClusterCoordinator clusterCoordinator; private VariableRegistry variableRegistry; + private LeaderElectionManager leaderElectionManager; @Override public Object getObject() throws Exception { @@ -69,7 +67,9 @@ public Object getObject() throws Exception { nodeProtocolSender, bulletinRepository, clusterCoordinator, - heartbeatMonitor, variableRegistry); + heartbeatMonitor, + leaderElectionManager, + variableRegistry); } else { flowController = FlowController.createStandaloneInstance( flowFileEventRepository, @@ -129,4 +129,8 @@ public void setVariableRegistry(VariableRegistry variableRegistry) { public void setClusterCoordinator(final ClusterCoordinator clusterCoordinator) { this.clusterCoordinator = clusterCoordinator; } + + public void setLeaderElectionManager(final LeaderElectionManager leaderElectionManager) { + this.leaderElectionManager = leaderElectionManager; + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/nifi-context.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/nifi-context.xml index f03e26c2e259..15032089c993 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/nifi-context.xml +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/nifi-context.xml @@ -47,6 +47,7 @@ + From 3fd121b2ff57c5fc51171791351580c29c58e553 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Sun, 14 Aug 2016 19:40:31 -0400 Subject: [PATCH 2/4] NIFI-2566: Removed storage of cluster roles from heartbeats and NodeConnectionStatus; use LeaderElectionManager to determine roles instead --- .../coordination/ClusterCoordinator.java | 26 +-- .../coordination/heartbeat/NodeHeartbeat.java | 7 - .../coordination/node/ClusterRoles.java | 10 ++ .../node/NodeConnectionStatus.java | 34 ++-- .../nifi/cluster/protocol/Heartbeat.java | 11 +- .../jaxb/message/AdaptedHeartbeat.java | 11 -- .../message/AdaptedNodeConnectionStatus.java | 11 -- .../jaxb/message/HeartbeatAdapter.java | 5 +- .../message/NodeConnectionStatusAdapter.java | 4 +- .../heartbeat/AbstractHeartbeatMonitor.java | 2 - .../ClusterProtocolHeartbeatMonitor.java | 100 +---------- .../heartbeat/StandardNodeHeartbeat.java | 15 +- .../node/NodeClusterCoordinator.java | 163 ++---------------- .../TestAbstractHeartbeatMonitor.java | 24 +-- .../TestClusterProtocolHeartbeatMonitor.java | 67 ------- .../TestThreadPoolRequestReplicator.java | 5 +- .../node/TestNodeClusterCoordinator.java | 93 ++-------- .../nifi/cluster/integration/Cluster.java | 4 +- .../integration/ClusterConnectionIT.java | 1 - .../apache/nifi/cluster/integration/Node.java | 23 ++- .../nifi/controller/FlowController.java | 22 +-- .../nifi/controller/StandardFlowService.java | 5 +- .../nifi/web/StandardNiFiServiceFacade.java | 42 +++-- .../main/resources/nifi-web-api-context.xml | 1 + 24 files changed, 125 insertions(+), 561 deletions(-) delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestClusterProtocolHeartbeatMonitor.java diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/ClusterCoordinator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/ClusterCoordinator.java index 4894fc539635..ff8d06ecafb7 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/ClusterCoordinator.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/ClusterCoordinator.java @@ -107,9 +107,9 @@ public interface ClusterCoordinator { Set getNodeIdentifiers(NodeConnectionState... states); /** - * Returns a Map of NodeConnectionStatus to all Node Identifiers that have that status. + * Returns a Map of NodeConnectionStates to all Node Identifiers that have that state. * - * @return the NodeConnectionStatus for each Node in the cluster, grouped by the Connection Status + * @return the NodeConnectionState for each Node in the cluster, grouped by the Connection State */ Map> getConnectionStates(); @@ -134,14 +134,6 @@ public interface ClusterCoordinator { */ void reportEvent(NodeIdentifier nodeId, Severity severity, String event); - /** - * Updates the roles held by the given node - * - * @param nodeId the id of the node to update - * @param roles the new roles that the node possesses - */ - void updateNodeRoles(NodeIdentifier nodeId, Set roles); - /** * Returns the NodeIdentifier that exists that has the given UUID, or null if no NodeIdentifier * exists for the given UUID @@ -216,18 +208,4 @@ public interface ClusterCoordinator { * @return true if connected, false otherwise */ boolean isConnected(); - - /** - * Notifies the cluster coordinator that this node has been granted the given role - * - * @param clusterRole the role that this node has been granted - */ - void addRole(String clusterRole); - - /** - * Notifies the cluster coordinator that this node is no longer responsible for the given role - * - * @param clusterRole the role that this node is no longer responsible for - */ - void removeRole(String clusterRole); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/NodeHeartbeat.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/NodeHeartbeat.java index 2ddda790121a..c4413a27c306 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/NodeHeartbeat.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/NodeHeartbeat.java @@ -17,8 +17,6 @@ package org.apache.nifi.cluster.coordination.heartbeat; -import java.util.Set; - import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; import org.apache.nifi.cluster.protocol.NodeIdentifier; @@ -39,11 +37,6 @@ public interface NodeHeartbeat { */ NodeConnectionStatus getConnectionStatus(); - /** - * @return the set of Roles that the node currently possesses. - */ - Set getRoles(); - /** * @return the number of FlowFiles that are queued up on the node */ diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/ClusterRoles.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/ClusterRoles.java index 611faa40f84d..1425c9dbfa24 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/ClusterRoles.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/ClusterRoles.java @@ -17,9 +17,19 @@ package org.apache.nifi.cluster.coordination.node; +import java.util.HashSet; +import java.util.Set; + public class ClusterRoles { public static final String PRIMARY_NODE = "Primary Node"; public static final String CLUSTER_COORDINATOR = "Cluster Coordinator"; + + public static Set getAllRoles() { + final Set roles = new HashSet<>(); + roles.add(PRIMARY_NODE); + roles.add(CLUSTER_COORDINATOR); + return roles; + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/NodeConnectionStatus.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/NodeConnectionStatus.java index 23e509dc8c9d..c3862ad1574d 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/NodeConnectionStatus.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/NodeConnectionStatus.java @@ -17,10 +17,7 @@ package org.apache.nifi.cluster.coordination.node; -import java.util.Collections; -import java.util.HashSet; import java.util.Objects; -import java.util.Set; import java.util.concurrent.atomic.AtomicLong; import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; @@ -41,40 +38,38 @@ public class NodeConnectionStatus { private final DisconnectionCode disconnectCode; private final String disconnectReason; private final Long connectionRequestTime; - private final Set roles; - public NodeConnectionStatus(final NodeIdentifier nodeId, final NodeConnectionState state, final Set roles) { - this(nodeId, state, null, null, null, roles); + public NodeConnectionStatus(final NodeIdentifier nodeId, final NodeConnectionState state) { + this(nodeId, state, null, null, null); } public NodeConnectionStatus(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode) { - this(nodeId, NodeConnectionState.DISCONNECTED, disconnectionCode, disconnectionCode.name(), null, null); + this(nodeId, NodeConnectionState.DISCONNECTED, disconnectionCode, disconnectionCode.toString(), null); } public NodeConnectionStatus(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode, final String disconnectionExplanation) { - this(nodeId, NodeConnectionState.DISCONNECTED, disconnectionCode, disconnectionExplanation, null, null); + this(nodeId, NodeConnectionState.DISCONNECTED, disconnectionCode, disconnectionExplanation, null); } - public NodeConnectionStatus(final NodeIdentifier nodeId, final NodeConnectionState state, final DisconnectionCode disconnectionCode, final Set roles) { - this(nodeId, state, disconnectionCode, disconnectionCode.name(), null, roles); + public NodeConnectionStatus(final NodeIdentifier nodeId, final NodeConnectionState state, final DisconnectionCode disconnectionCode) { + this(nodeId, state, disconnectionCode, disconnectionCode.toString(), null); } - public NodeConnectionStatus(final NodeConnectionStatus status, final Set roles) { - this(status.getNodeIdentifier(), status.getState(), status.getDisconnectCode(), status.getDisconnectReason(), status.getConnectionRequestTime(), roles); + public NodeConnectionStatus(final NodeConnectionStatus status) { + this(status.getNodeIdentifier(), status.getState(), status.getDisconnectCode(), status.getDisconnectReason(), status.getConnectionRequestTime()); } public NodeConnectionStatus(final NodeIdentifier nodeId, final NodeConnectionState state, final DisconnectionCode disconnectCode, - final String disconnectReason, final Long connectionRequestTime, final Set roles) { - this(idGenerator.getAndIncrement(), nodeId, state, disconnectCode, disconnectReason, connectionRequestTime, roles); + final String disconnectReason, final Long connectionRequestTime) { + this(idGenerator.getAndIncrement(), nodeId, state, disconnectCode, disconnectReason, connectionRequestTime); } public NodeConnectionStatus(final long updateId, final NodeIdentifier nodeId, final NodeConnectionState state, final DisconnectionCode disconnectCode, - final String disconnectReason, final Long connectionRequestTime, final Set roles) { + final String disconnectReason, final Long connectionRequestTime) { this.updateId = updateId; this.nodeId = nodeId; this.state = state; - this.roles = roles == null ? Collections.emptySet() : Collections.unmodifiableSet(new HashSet<>(roles)); if (state == NodeConnectionState.DISCONNECTED && disconnectCode == null) { this.disconnectCode = DisconnectionCode.UNKNOWN; this.disconnectReason = this.disconnectCode.toString(); @@ -90,10 +85,6 @@ public long getUpdateIdentifier() { return updateId; } - public Set getRoles() { - return roles; - } - public NodeIdentifier getNodeIdentifier() { return nodeId; } @@ -122,7 +113,6 @@ public String toString() { if (state == NodeConnectionState.DISCONNECTED || state == NodeConnectionState.DISCONNECTING) { sb.append(", Disconnect Code=").append(getDisconnectCode()).append(", Disconnect Reason=").append(getDisconnectReason()); } - sb.append(", roles=").append(getRoles()); sb.append(", updateId=").append(getUpdateIdentifier()); sb.append("]"); return sb.toString(); @@ -142,7 +132,6 @@ public int hashCode() { final int prime = 31; int result = 1; result = prime * result + ((nodeId == null) ? 0 : nodeId.hashCode()); - result = prime * result + ((roles == null) ? 0 : roles.hashCode()); result = prime * result + ((state == null) ? 0 : state.hashCode()); return result; } @@ -163,7 +152,6 @@ public boolean equals(Object obj) { NodeConnectionStatus other = (NodeConnectionStatus) obj; return Objects.deepEquals(getNodeIdentifier(), other.getNodeIdentifier()) - && Objects.deepEquals(getRoles(), other.getRoles()) && Objects.deepEquals(getState(), other.getState()); } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/Heartbeat.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/Heartbeat.java index 2135f20dc05e..a028c8e6bb27 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/Heartbeat.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/Heartbeat.java @@ -16,10 +16,7 @@ */ package org.apache.nifi.cluster.protocol; -import java.util.Collections; import java.util.Date; -import java.util.HashSet; -import java.util.Set; import javax.xml.bind.annotation.XmlTransient; import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; @@ -35,17 +32,15 @@ public class Heartbeat { private final NodeIdentifier nodeIdentifier; - private final Set roles; private final NodeConnectionStatus connectionStatus; private final long createdTimestamp; private final byte[] payload; - public Heartbeat(final NodeIdentifier nodeIdentifier, final Set roles, final NodeConnectionStatus connectionStatus, final byte[] payload) { + public Heartbeat(final NodeIdentifier nodeIdentifier, final NodeConnectionStatus connectionStatus, final byte[] payload) { if (nodeIdentifier == null) { throw new IllegalArgumentException("Node Identifier may not be null."); } this.nodeIdentifier = nodeIdentifier; - this.roles = roles == null ? Collections.emptySet() : Collections.unmodifiableSet(new HashSet<>(roles)); this.connectionStatus = connectionStatus; this.payload = payload; this.createdTimestamp = new Date().getTime(); @@ -59,10 +54,6 @@ public byte[] getPayload() { return payload; } - public Set getRoles() { - return roles; - } - public NodeConnectionStatus getConnectionStatus() { return connectionStatus; } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedHeartbeat.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedHeartbeat.java index f1eba521c507..6f718fa5cffa 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedHeartbeat.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedHeartbeat.java @@ -16,8 +16,6 @@ */ package org.apache.nifi.cluster.protocol.jaxb.message; -import java.util.Set; - import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; @@ -29,7 +27,6 @@ public class AdaptedHeartbeat { private NodeIdentifier nodeIdentifier; private byte[] payload; - private Set roles; private NodeConnectionStatus connectionStatus; public AdaptedHeartbeat() { @@ -44,14 +41,6 @@ public void setNodeIdentifier(NodeIdentifier nodeIdentifier) { this.nodeIdentifier = nodeIdentifier; } - public Set getRoles() { - return roles; - } - - public void setRoles(Set roles) { - this.roles = roles; - } - public void setConnectionStatus(NodeConnectionStatus connectionStatus) { this.connectionStatus = connectionStatus; } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeConnectionStatus.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeConnectionStatus.java index 9cfac2cc7bc1..c8c4acf646a3 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeConnectionStatus.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeConnectionStatus.java @@ -17,8 +17,6 @@ package org.apache.nifi.cluster.protocol.jaxb.message; -import java.util.Set; - import org.apache.nifi.cluster.coordination.node.DisconnectionCode; import org.apache.nifi.cluster.coordination.node.NodeConnectionState; import org.apache.nifi.cluster.protocol.NodeIdentifier; @@ -30,7 +28,6 @@ public class AdaptedNodeConnectionStatus { private DisconnectionCode disconnectCode; private String disconnectReason; private Long connectionRequestTime; - private Set roles; public Long getUpdateId() { return updateId; @@ -79,12 +76,4 @@ public Long getConnectionRequestTime() { public void setConnectionRequestTime(Long connectionRequestTime) { this.connectionRequestTime = connectionRequestTime; } - - public Set getRoles() { - return roles; - } - - public void setRoles(Set roles) { - this.roles = roles; - } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/HeartbeatAdapter.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/HeartbeatAdapter.java index 94d26ce3751c..58e675bf406e 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/HeartbeatAdapter.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/HeartbeatAdapter.java @@ -34,9 +34,6 @@ public AdaptedHeartbeat marshal(final Heartbeat hb) { // set payload aHb.setPayload(hb.getPayload()); - // set leader flag - aHb.setRoles(hb.getRoles()); - // set connected flag aHb.setConnectionStatus(hb.getConnectionStatus()); } @@ -46,7 +43,7 @@ public AdaptedHeartbeat marshal(final Heartbeat hb) { @Override public Heartbeat unmarshal(final AdaptedHeartbeat aHb) { - return new Heartbeat(aHb.getNodeIdentifier(), aHb.getRoles(), aHb.getConnectionStatus(), aHb.getPayload()); + return new Heartbeat(aHb.getNodeIdentifier(), aHb.getConnectionStatus(), aHb.getPayload()); } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeConnectionStatusAdapter.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeConnectionStatusAdapter.java index 21d0bdab57be..ec209de1f540 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeConnectionStatusAdapter.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeConnectionStatusAdapter.java @@ -30,8 +30,7 @@ public NodeConnectionStatus unmarshal(final AdaptedNodeConnectionStatus adapted) adapted.getState(), adapted.getDisconnectCode(), adapted.getDisconnectReason(), - adapted.getConnectionRequestTime(), - adapted.getRoles()); + adapted.getConnectionRequestTime()); } @Override @@ -44,7 +43,6 @@ public AdaptedNodeConnectionStatus marshal(final NodeConnectionStatus toAdapt) t adapted.setDisconnectCode(toAdapt.getDisconnectCode()); adapted.setDisconnectReason(toAdapt.getDisconnectReason()); adapted.setState(toAdapt.getState()); - adapted.setRoles(toAdapt.getRoles()); } return adapted; } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/AbstractHeartbeatMonitor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/AbstractHeartbeatMonitor.java index c216ed32c29a..e304c23c7b90 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/AbstractHeartbeatMonitor.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/AbstractHeartbeatMonitor.java @@ -256,8 +256,6 @@ private void processHeartbeat(final NodeHeartbeat heartbeat) { clusterCoordinator.finishNodeConnection(nodeId); clusterCoordinator.reportEvent(nodeId, Severity.INFO, "Received first heartbeat from connecting node. Node connected."); } - - clusterCoordinator.updateNodeRoles(nodeId, heartbeat.getRoles()); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/ClusterProtocolHeartbeatMonitor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/ClusterProtocolHeartbeatMonitor.java index 5ecd8cbd17da..3afd6a63b82d 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/ClusterProtocolHeartbeatMonitor.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/ClusterProtocolHeartbeatMonitor.java @@ -17,25 +17,15 @@ package org.apache.nifi.cluster.coordination.heartbeat; -import java.nio.charset.StandardCharsets; import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; import java.util.Map; import java.util.Properties; -import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import javax.xml.bind.JAXBContext; import javax.xml.bind.Unmarshaller; -import org.apache.curator.RetryPolicy; -import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.framework.CuratorFrameworkFactory; -import org.apache.curator.framework.state.ConnectionState; -import org.apache.curator.framework.state.ConnectionStateListener; -import org.apache.curator.retry.RetryForever; import org.apache.nifi.cluster.HeartbeatPayload; import org.apache.nifi.cluster.coordination.ClusterCoordinator; import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; @@ -47,11 +37,7 @@ import org.apache.nifi.cluster.protocol.message.HeartbeatMessage; import org.apache.nifi.cluster.protocol.message.ProtocolMessage; import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType; -import org.apache.nifi.controller.cluster.ZooKeeperClientConfig; import org.apache.nifi.util.NiFiProperties; -import org.apache.zookeeper.CreateMode; -import org.apache.zookeeper.KeeperException.NoNodeException; -import org.apache.zookeeper.KeeperException.NodeExistsException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -59,15 +45,8 @@ * Uses Apache ZooKeeper to advertise the address to send heartbeats to, and then relies on the NiFi Cluster * Protocol to receive heartbeat messages from nodes in the cluster. */ -public class ClusterProtocolHeartbeatMonitor extends AbstractHeartbeatMonitor implements HeartbeatMonitor, ProtocolHandler, ConnectionStateListener { +public class ClusterProtocolHeartbeatMonitor extends AbstractHeartbeatMonitor implements HeartbeatMonitor, ProtocolHandler { protected static final Logger logger = LoggerFactory.getLogger(ClusterProtocolHeartbeatMonitor.class); - private static final String COORDINATOR_ZNODE_NAME = "coordinator"; - - private final ZooKeeperClientConfig zkClientConfig; - private final String clusterNodesPath; - - private volatile Map clusterNodeIds = new HashMap<>(); - private volatile CuratorFramework curatorClient; private final String heartbeatAddress; private final ConcurrentMap heartbeatMessages = new ConcurrentHashMap<>(); @@ -88,8 +67,6 @@ public ClusterProtocolHeartbeatMonitor(final ClusterCoordinator clusterCoordinat super(clusterCoordinator, properties); protocolListener.addHandler(this); - this.zkClientConfig = ZooKeeperClientConfig.createConfig(properties); - this.clusterNodesPath = zkClientConfig.resolvePath("cluster/nodes"); String hostname = properties.getProperty(NiFiProperties.CLUSTER_NODE_ADDRESS); if (hostname == null || hostname.trim().isEmpty()) { @@ -119,17 +96,6 @@ public String getHeartbeatAddress() { @Override public void onStart() { - final RetryPolicy retryPolicy = new RetryForever(5000); - curatorClient = CuratorFrameworkFactory.builder() - .connectString(zkClientConfig.getConnectString()) - .sessionTimeoutMs(zkClientConfig.getSessionTimeoutMillis()) - .connectionTimeoutMs(zkClientConfig.getConnectionTimeoutMillis()) - .retryPolicy(retryPolicy) - .defaultData(new byte[0]) - .build(); - curatorClient.getConnectionStateListenable().addListener(this); - curatorClient.start(); - // We don't know what the heartbeats look like for the nodes, since we were just elected to monitoring // them. However, the map may be filled with old heartbeats. So we clear the heartbeats and populate the // map with new heartbeats set to the current time and using the currently known status. We do this so @@ -138,16 +104,13 @@ public void onStart() { heartbeatMessages.clear(); for (final NodeIdentifier nodeId : clusterCoordinator.getNodeIdentifiers()) { final NodeHeartbeat heartbeat = new StandardNodeHeartbeat(nodeId, System.currentTimeMillis(), - clusterCoordinator.getConnectionStatus(nodeId), Collections.emptySet(), 0, 0L, 0, System.currentTimeMillis()); + clusterCoordinator.getConnectionStatus(nodeId), 0, 0L, 0, System.currentTimeMillis()); heartbeatMessages.put(nodeId, heartbeat); } } @Override public void onStop() { - if (curatorClient != null) { - curatorClient.close(); - } } @Override @@ -161,10 +124,6 @@ public synchronized void removeHeartbeat(final NodeIdentifier nodeId) { heartbeatMessages.remove(nodeId); } - protected Set getClusterNodeIds() { - return new HashSet<>(clusterNodeIds.values()); - } - @Override public ProtocolMessage handle(final ProtocolMessage msg) throws ProtocolException { @@ -177,7 +136,6 @@ public ProtocolMessage handle(final ProtocolMessage msg) throws ProtocolExceptio final NodeIdentifier nodeId = heartbeat.getNodeIdentifier(); final NodeConnectionStatus connectionStatus = heartbeat.getConnectionStatus(); - final Set roles = heartbeat.getRoles(); final byte[] payloadBytes = heartbeat.getPayload(); final HeartbeatPayload payload = HeartbeatPayload.unmarshal(payloadBytes); final int activeThreadCount = payload.getActiveThreadCount(); @@ -186,7 +144,7 @@ public ProtocolMessage handle(final ProtocolMessage msg) throws ProtocolExceptio final long systemStartTime = payload.getSystemStartTime(); final NodeHeartbeat nodeHeartbeat = new StandardNodeHeartbeat(nodeId, System.currentTimeMillis(), - connectionStatus, roles, flowFileCount, flowFileBytes, activeThreadCount, systemStartTime); + connectionStatus, flowFileCount, flowFileBytes, activeThreadCount, systemStartTime); heartbeatMessages.put(heartbeat.getNodeIdentifier(), nodeHeartbeat); logger.debug("Received new heartbeat from {}", nodeId); @@ -197,56 +155,4 @@ public ProtocolMessage handle(final ProtocolMessage msg) throws ProtocolExceptio public boolean canHandle(ProtocolMessage msg) { return msg.getType() == MessageType.HEARTBEAT; } - - protected void publishAddressAsync() { - final Thread publishAddressThread = new Thread(() -> publishAddress()); - - publishAddressThread.setName("Publish Heartbeat Address"); - publishAddressThread.setDaemon(true); - publishAddressThread.start(); - } - - protected void publishAddress() { - if (true) { - return; - } - while (!isStopped()) { - final String path = clusterNodesPath + "/" + COORDINATOR_ZNODE_NAME; - try { - try { - curatorClient.setData().forPath(path, heartbeatAddress.getBytes(StandardCharsets.UTF_8)); - logger.info("Successfully published Cluster Heartbeat Monitor Address of {} to ZooKeeper", heartbeatAddress); - return; - } catch (final NoNodeException nne) { - // ensure that parents are created, using a wide-open ACL because the parents contain no data - // and the path is not in any way sensitive. - try { - curatorClient.create().creatingParentsIfNeeded().withMode(CreateMode.EPHEMERAL).forPath(path, heartbeatAddress.getBytes(StandardCharsets.UTF_8)); - } catch (final NodeExistsException nee) { - // This is okay. Node already exists. - } - - logger.info("Successfully published address as heartbeat monitor address at path {} with value {}", path, heartbeatAddress); - - return; - } - } catch (Exception e) { - logger.warn("Failed to update ZooKeeper to notify nodes of the heartbeat address. Will continue to retry."); - - try { - Thread.sleep(2000L); - } catch (final InterruptedException ie) { - Thread.currentThread().interrupt(); - return; - } - } - } - } - - @Override - public void stateChanged(final CuratorFramework client, final ConnectionState newState) { - if (newState.isConnected()) { - publishAddressAsync(); - } - } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/StandardNodeHeartbeat.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/StandardNodeHeartbeat.java index e455a76db993..8fec4cf0cf2c 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/StandardNodeHeartbeat.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/StandardNodeHeartbeat.java @@ -17,10 +17,6 @@ package org.apache.nifi.cluster.coordination.heartbeat; -import java.util.Collections; -import java.util.HashSet; -import java.util.Set; - import org.apache.nifi.cluster.HeartbeatPayload; import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; import org.apache.nifi.cluster.protocol.Heartbeat; @@ -32,18 +28,16 @@ public class StandardNodeHeartbeat implements NodeHeartbeat { private final NodeIdentifier nodeId; private final long timestamp; private final NodeConnectionStatus connectionStatus; - private final Set roles; private final int flowFileCount; private final long flowFileBytes; private final int activeThreadCount; private final long systemStartTime; public StandardNodeHeartbeat(final NodeIdentifier nodeId, final long timestamp, final NodeConnectionStatus connectionStatus, - final Set roles, final int flowFileCount, final long flowFileBytes, final int activeThreadCount, final long systemStartTime) { + final int flowFileCount, final long flowFileBytes, final int activeThreadCount, final long systemStartTime) { this.timestamp = timestamp; this.nodeId = nodeId; this.connectionStatus = connectionStatus; - this.roles = roles == null ? Collections.emptySet() : Collections.unmodifiableSet(new HashSet<>(roles)); this.flowFileCount = flowFileCount; this.flowFileBytes = flowFileBytes; this.activeThreadCount = activeThreadCount; @@ -65,11 +59,6 @@ public NodeConnectionStatus getConnectionStatus() { return connectionStatus; } - @Override - public Set getRoles() { - return roles; - } - @Override public int getFlowFileCount() { return flowFileCount; @@ -96,7 +85,7 @@ public static StandardNodeHeartbeat fromHeartbeatMessage(final HeartbeatMessage final HeartbeatPayload payload = HeartbeatPayload.unmarshal(heartbeat.getPayload()); return new StandardNodeHeartbeat(heartbeat.getNodeIdentifier(), timestamp, heartbeat.getConnectionStatus(), - heartbeat.getRoles(), (int) payload.getTotalFlowFileCount(), payload.getTotalFlowFileBytes(), + (int) payload.getTotalFlowFileCount(), payload.getTotalFlowFileBytes(), payload.getActiveThreadCount(), payload.getSystemStartTime()); } } \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java index 67e74614cf27..faea59fd6960 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java @@ -44,7 +44,6 @@ import org.apache.nifi.cluster.manager.NodeResponse; import org.apache.nifi.cluster.manager.exception.IllegalNodeDisconnectionException; import org.apache.nifi.cluster.manager.exception.NoClusterCoordinatorException; -import org.apache.nifi.cluster.manager.exception.UnknownNodeException; import org.apache.nifi.cluster.protocol.ComponentRevision; import org.apache.nifi.cluster.protocol.ConnectionRequest; import org.apache.nifi.cluster.protocol.ConnectionResponse; @@ -129,10 +128,6 @@ public NodeIdentifier getLocalNodeIdentifier() { return nodeId; } - private NodeIdentifier waitForLocalNodeIdentifier() { - return waitForNodeIdentifier(() -> getLocalNodeIdentifier()); - } - private NodeIdentifier waitForElectedClusterCoordinator() { return waitForNodeIdentifier(() -> getElectedActiveCoordinatorNode(false)); } @@ -212,7 +207,7 @@ public void requestNodeConnect(final NodeIdentifier nodeId, final String userDn) reportEvent(nodeId, Severity.INFO, "Requesting that node connect to cluster on behalf of " + userDn); } - updateNodeStatus(new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTING, null, null, System.currentTimeMillis(), getRoles(nodeId))); + updateNodeStatus(new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTING, null, null, System.currentTimeMillis())); // create the request final ReconnectionRequestMessage request = new ReconnectionRequestMessage(); @@ -222,10 +217,6 @@ public void requestNodeConnect(final NodeIdentifier nodeId, final String userDn) requestReconnectionAsynchronously(request, 10, 5); } - private Set getRoles(final NodeIdentifier nodeId) { - final NodeConnectionStatus status = getConnectionStatus(nodeId); - return status == null ? Collections.emptySet() : status.getRoles(); - } @Override public void finishNodeConnection(final NodeIdentifier nodeId) { @@ -248,7 +239,7 @@ public void finishNodeConnection(final NodeIdentifier nodeId) { } logger.info("{} is now connected", nodeId); - updateNodeStatus(new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTED, getRoles(nodeId))); + updateNodeStatus(new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTED)); } @@ -305,7 +296,7 @@ public void removeNode(final NodeIdentifier nodeId, final String userDn) { reportEvent(nodeId, Severity.INFO, "User " + userDn + " requested that node be removed from cluster"); nodeStatuses.remove(nodeId); nodeEvents.remove(nodeId); - notifyOthersOfNodeStatusChange(new NodeConnectionStatus(nodeId, NodeConnectionState.REMOVED, Collections.emptySet())); + notifyOthersOfNodeStatusChange(new NodeConnectionStatus(nodeId, NodeConnectionState.REMOVED)); } @Override @@ -357,57 +348,6 @@ public void reportEvent(final NodeIdentifier nodeId, final Severity severity, fi } } - @Override - public synchronized void updateNodeRoles(final NodeIdentifier nodeId, final Set roles) { - boolean updated = false; - while (!updated) { - final NodeConnectionStatus currentStatus = nodeStatuses.get(nodeId); - if (currentStatus == null) { - throw new UnknownNodeException("Cannot update roles for " + nodeId + " to " + roles + " because the node is not part of this cluster"); - } - - if (currentStatus.getRoles().equals(roles)) { - logger.debug("Roles for {} already up-to-date as {}", nodeId, roles); - return; - } - - final NodeConnectionStatus updatedStatus = new NodeConnectionStatus(currentStatus, roles); - updated = replaceNodeStatus(nodeId, currentStatus, updatedStatus); - - if (updated) { - logger.info("Updated Roles of {} from {} to {}", nodeId, currentStatus, updatedStatus); - notifyOthersOfNodeStatusChange(updatedStatus); - } - } - - // If any other node contains any of the given roles, revoke the role from the other node. - for (final String role : roles) { - for (final Map.Entry entry : nodeStatuses.entrySet()) { - if (entry.getKey().equals(nodeId)) { - continue; - } - - updated = false; - while (!updated) { - final NodeConnectionStatus status = entry.getValue(); - if (status.getRoles().contains(role)) { - final Set newRoles = new HashSet<>(status.getRoles()); - newRoles.remove(role); - - final NodeConnectionStatus updatedStatus = new NodeConnectionStatus(status, newRoles); - updated = replaceNodeStatus(entry.getKey(), status, updatedStatus); - - if (updated) { - logger.info("Updated Roles of {} from {} to {}", nodeId, status, updatedStatus); - notifyOthersOfNodeStatusChange(updatedStatus); - } - } else { - updated = true; - } - } - } - } - } @Override public NodeIdentifier getNodeIdentifier(final String uuid) { @@ -420,47 +360,6 @@ public NodeIdentifier getNodeIdentifier(final String uuid) { return null; } - // method is synchronized because it modifies local node state and then broadcasts the change. We synchronize any time that this - // is done so that we don't have an issue where we create a NodeConnectionStatus, then another thread creates one and sends it - // before the first one is sent (as this results in the first status having a larger id, which means that the first status is never - // seen by other nodes). - @Override - public synchronized void addRole(final String clusterRole) { - final NodeIdentifier localNodeId = waitForLocalNodeIdentifier(); - final NodeConnectionStatus status = getConnectionStatus(localNodeId); - final Set roles = new HashSet<>(); - if (status != null) { - roles.addAll(status.getRoles()); - } - - final boolean roleAdded = roles.add(clusterRole); - - if (roleAdded) { - updateNodeRoles(localNodeId, roles); - logger.info("Cluster role {} added. This node is now responsible for the following roles: {}", clusterRole, roles); - } - } - - // method is synchronized because it modifies local node state and then broadcasts the change. We synchronize any time that this - // is done so that we don't have an issue where we create a NodeConnectionStatus, then another thread creates one and sends it - // before the first one is sent (as this results in the first status having a larger id, which means that the first status is never - // seen by other nodes). - @Override - public synchronized void removeRole(final String clusterRole) { - final NodeIdentifier localNodeId = waitForLocalNodeIdentifier(); - final NodeConnectionStatus status = getConnectionStatus(localNodeId); - final Set roles = new HashSet<>(); - if (status != null) { - roles.addAll(status.getRoles()); - } - - final boolean roleRemoved = roles.remove(clusterRole); - - if (roleRemoved) { - updateNodeRoles(localNodeId, roles); - logger.info("Cluster role {} removed. This node is now responsible for the following roles: {}", clusterRole, roles); - } - } @Override public Set getNodeIdentifiers(final NodeConnectionState... states) { @@ -483,10 +382,14 @@ public Set getNodeIdentifiers(final NodeConnectionState... state @Override public NodeIdentifier getPrimaryNode() { - return nodeStatuses.values().stream() - .filter(status -> status.getRoles().contains(ClusterRoles.PRIMARY_NODE)) + final String primaryNodeAddress = leaderElectionManager.getLeader(ClusterRoles.PRIMARY_NODE); + if (primaryNodeAddress == null) { + return null; + } + + return nodeStatuses.keySet().stream() + .filter(nodeId -> primaryNodeAddress.equals(nodeId.getSocketAddress() + ":" + nodeId.getSocketPort())) .findFirst() - .map(status -> status.getNodeIdentifier()) .orElse(null); } @@ -640,11 +543,7 @@ void updateNodeStatus(final NodeConnectionStatus status, final boolean waitForCo logger.debug("State of cluster nodes is now {}", nodeStatuses); if (currentState == null || currentState != status.getState()) { - // We notify all nodes of the status change if either this node is the current cluster coordinator, OR if the node was - // the cluster coordinator and no longer is. This is done because if a user disconnects the cluster coordinator, we need - // to broadcast to the cluster that this node is no longer the coordinator. Otherwise, all nodes but this one will still - // believe that this node is connected to the cluster. - final boolean notifyAllNodes = isActiveClusterCoordinator() || (currentStatus != null && currentStatus.getRoles().contains(ClusterRoles.CLUSTER_COORDINATOR)); + final boolean notifyAllNodes = isActiveClusterCoordinator(); if (notifyAllNodes) { logger.debug("Notifying all nodes that status changed from {} to {}", currentStatus, status); } else { @@ -803,43 +702,9 @@ private NodeConnectionStatusResponseMessage handleNodeConnectionStatusRequest() private String summarizeStatusChange(final NodeConnectionStatus oldStatus, final NodeConnectionStatus status) { final StringBuilder sb = new StringBuilder(); - if (oldStatus != null && status.getState() == oldStatus.getState()) { - // Check if roles changed - final Set oldRoles = oldStatus.getRoles(); - final Set newRoles = status.getRoles(); - - final Set rolesRemoved = new HashSet<>(oldRoles); - rolesRemoved.removeAll(newRoles); - - final Set rolesAdded = new HashSet<>(newRoles); - rolesAdded.removeAll(oldRoles); - - if (!rolesRemoved.isEmpty()) { - sb.append("Relinquished role"); - if (rolesRemoved.size() != 1) { - sb.append("s"); - } - - sb.append(" ").append(rolesRemoved); - } - - if (!rolesAdded.isEmpty()) { - if (sb.length() > 0) { - sb.append("; "); - } - - sb.append("Acquired role"); - if (rolesAdded.size() != 1) { - sb.append("s"); - } - - sb.append(" ").append(rolesAdded); - } - } else { + if (oldStatus == null || status.getState() != oldStatus.getState()) { sb.append("Node Status changed from ").append(oldStatus == null ? "[Unknown Node]" : oldStatus.getState().toString()).append(" to ").append(status.getState().toString()); - if (status.getState() == NodeConnectionState.CONNECTED) { - sb.append(" (Roles=").append(status.getRoles().toString()).append(")"); - } else if (status.getDisconnectReason() != null) { + if (status.getDisconnectReason() != null) { sb.append(" due to ").append(status.getDisconnectReason()); } else if (status.getDisconnectCode() != null) { sb.append(" due to ").append(status.getDisconnectCode().toString()); @@ -940,7 +805,7 @@ private ConnectionResponse createConnectionResponse(final ConnectionRequest requ addNodeEvent(resolvedNodeIdentifier, "Connection requested from existing node. Setting status to connecting"); } - status = new NodeConnectionStatus(resolvedNodeIdentifier, NodeConnectionState.CONNECTING, null, null, System.currentTimeMillis(), getRoles(resolvedNodeIdentifier)); + status = new NodeConnectionStatus(resolvedNodeIdentifier, NodeConnectionState.CONNECTING, null, null, System.currentTimeMillis()); updateNodeStatus(status); DataFlow dataFlow = null; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java index bb910a5bb6c1..06bae42868c3 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java @@ -169,8 +169,8 @@ public boolean isActiveClusterCoordinator() { private NodeHeartbeat createHeartbeat(final NodeIdentifier nodeId, final NodeConnectionState state) { - final NodeConnectionStatus status = new NodeConnectionStatus(nodeId, state, Collections.emptySet()); - return new StandardNodeHeartbeat(nodeId, System.currentTimeMillis(), status, Collections.emptySet(), 0, 0, 0, 0); + final NodeConnectionStatus status = new NodeConnectionStatus(nodeId, state); + return new StandardNodeHeartbeat(nodeId, System.currentTimeMillis(), status, 0, 0, 0, 0); } private TestFriendlyHeartbeatMonitor createMonitor(final ClusterCoordinator coordinator) { @@ -195,7 +195,7 @@ public synchronized void requestNodeConnect(NodeIdentifier nodeId) { @Override public synchronized void requestNodeConnect(NodeIdentifier nodeId, String userDn) { - statuses.put(nodeId, new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTING, Collections.emptySet())); + statuses.put(nodeId, new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTING)); } @Override @@ -205,17 +205,17 @@ public void removeNode(NodeIdentifier nodeId, String userDn) { @Override public synchronized void finishNodeConnection(NodeIdentifier nodeId) { - statuses.put(nodeId, new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTED, Collections.emptySet())); + statuses.put(nodeId, new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTED)); } @Override public synchronized void requestNodeDisconnect(NodeIdentifier nodeId, DisconnectionCode disconnectionCode, String explanation) { - statuses.put(nodeId, new NodeConnectionStatus(nodeId, NodeConnectionState.DISCONNECTED, Collections.emptySet())); + statuses.put(nodeId, new NodeConnectionStatus(nodeId, NodeConnectionState.DISCONNECTED)); } @Override public synchronized void disconnectionRequestedByNode(NodeIdentifier nodeId, DisconnectionCode disconnectionCode, String explanation) { - statuses.put(nodeId, new NodeConnectionStatus(nodeId, NodeConnectionState.DISCONNECTED, Collections.emptySet())); + statuses.put(nodeId, new NodeConnectionStatus(nodeId, NodeConnectionState.DISCONNECTED)); } @Override @@ -246,10 +246,6 @@ public synchronized void reportEvent(NodeIdentifier nodeId, Severity severity, S events.add(new ReportedEvent(nodeId, severity, event)); } - @Override - public void updateNodeRoles(NodeIdentifier nodeId, Set roles) { - } - synchronized List getEvents() { return new ArrayList<>(events); } @@ -309,14 +305,6 @@ public boolean isActiveClusterCoordinator() { return false; } - @Override - public void addRole(String clusterRole) { - } - - @Override - public void removeRole(String clusterRole) { - } - @Override public NodeIdentifier getLocalNodeIdentifier() { return null; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestClusterProtocolHeartbeatMonitor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestClusterProtocolHeartbeatMonitor.java deleted file mode 100644 index 1d95439c7c84..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestClusterProtocolHeartbeatMonitor.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * 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.nifi.cluster.coordination.heartbeat; - -import static org.junit.Assert.assertEquals; - -import java.util.Properties; -import java.util.concurrent.atomic.AtomicInteger; - -import org.apache.curator.framework.state.ConnectionState; -import org.apache.nifi.cluster.coordination.ClusterCoordinator; -import org.apache.nifi.cluster.protocol.ProtocolListener; -import org.apache.nifi.util.NiFiProperties; -import org.junit.Test; -import org.mockito.Mockito; - -public class TestClusterProtocolHeartbeatMonitor { - - @Test - public void testRepublishesAddressOnZooKeeperReconnect() throws InterruptedException { - final ClusterCoordinator coordinator = Mockito.mock(ClusterCoordinator.class); - final ProtocolListener protocolListener = Mockito.mock(ProtocolListener.class); - final Properties properties = new Properties(); - properties.setProperty(NiFiProperties.CLUSTER_NODE_PROTOCOL_PORT, "0"); - properties.setProperty(NiFiProperties.ZOOKEEPER_CONNECT_STRING, "localhost:2181"); - - final AtomicInteger publishCount = new AtomicInteger(0); - final ClusterProtocolHeartbeatMonitor monitor = new ClusterProtocolHeartbeatMonitor(coordinator, protocolListener, properties) { - @Override - protected void publishAddress() { - publishCount.incrementAndGet(); - } - }; - - monitor.start(); - - Thread.sleep(250L); - assertEquals(0, publishCount.get()); - monitor.stateChanged(null, ConnectionState.CONNECTED); - - Thread.sleep(250L); - assertEquals(1, publishCount.get()); - - monitor.stateChanged(null, ConnectionState.LOST); - Thread.sleep(250L); - assertEquals(1, publishCount.get()); - - monitor.stateChanged(null, ConnectionState.CONNECTED); - Thread.sleep(250L); - assertEquals(2, publishCount.get()); - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/replication/TestThreadPoolRequestReplicator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/replication/TestThreadPoolRequestReplicator.java index 5eac84616caa..f18d5890d33b 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/replication/TestThreadPoolRequestReplicator.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/replication/TestThreadPoolRequestReplicator.java @@ -28,7 +28,6 @@ import java.net.URI; import java.net.URISyntaxException; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -164,7 +163,7 @@ public void testMultipleRequestWithTwoPhaseCommit() { nodeIds.add(nodeId); final ClusterCoordinator coordinator = Mockito.mock(ClusterCoordinator.class); - Mockito.when(coordinator.getConnectionStatus(Mockito.any(NodeIdentifier.class))).thenReturn(new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTED, Collections.emptySet())); + Mockito.when(coordinator.getConnectionStatus(Mockito.any(NodeIdentifier.class))).thenReturn(new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTED)); final AtomicInteger requestCount = new AtomicInteger(0); final ThreadPoolRequestReplicator replicator = new ThreadPoolRequestReplicator(2, new Client(), coordinator, "1 sec", "1 sec", null, null) { @@ -210,7 +209,7 @@ private ClusterCoordinator createClusterCoordinator() { Mockito.when(coordinator.getConnectionStatus(Mockito.any(NodeIdentifier.class))).thenAnswer(new Answer() { @Override public NodeConnectionStatus answer(InvocationOnMock invocation) throws Throwable { - return new NodeConnectionStatus(invocation.getArgumentAt(0, NodeIdentifier.class), NodeConnectionState.CONNECTED, Collections.emptySet()); + return new NodeConnectionStatus(invocation.getArgumentAt(0, NodeIdentifier.class), NodeConnectionState.CONNECTED); } }); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/node/TestNodeClusterCoordinator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/node/TestNodeClusterCoordinator.java index 9f6928942d01..91174cabdda6 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/node/TestNodeClusterCoordinator.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/node/TestNodeClusterCoordinator.java @@ -86,10 +86,10 @@ void notifyOthersOfNodeStatusChange(NodeConnectionStatus updatedStatus, boolean public void testConnectionResponseIndicatesAllNodes() throws IOException { // Add a disconnected node coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(1), DisconnectionCode.LACK_OF_HEARTBEAT)); - coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(2), NodeConnectionState.DISCONNECTING, Collections.emptySet())); - coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(3), NodeConnectionState.CONNECTING, Collections.emptySet())); - coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(4), NodeConnectionState.CONNECTED, Collections.emptySet())); - coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(5), NodeConnectionState.CONNECTED, Collections.emptySet())); + coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(2), NodeConnectionState.DISCONNECTING)); + coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(3), NodeConnectionState.CONNECTING)); + coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(4), NodeConnectionState.CONNECTED)); + coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(5), NodeConnectionState.CONNECTED)); // Create a connection request message and send to the coordinator final NodeIdentifier requestedNodeId = createNodeId(6); @@ -257,10 +257,10 @@ public void testStatusChangesReplicated() throws InterruptedException, IOExcepti public void testGetConnectionStates() throws IOException { // Add a disconnected node coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(1), DisconnectionCode.LACK_OF_HEARTBEAT)); - coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(2), NodeConnectionState.DISCONNECTING, Collections.emptySet())); - coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(3), NodeConnectionState.CONNECTING, Collections.emptySet())); - coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(4), NodeConnectionState.CONNECTED, Collections.emptySet())); - coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(5), NodeConnectionState.CONNECTED, Collections.emptySet())); + coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(2), NodeConnectionState.DISCONNECTING)); + coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(3), NodeConnectionState.CONNECTING)); + coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(4), NodeConnectionState.CONNECTED)); + coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(5), NodeConnectionState.CONNECTED)); final Map> stateMap = coordinator.getConnectionStates(); assertEquals(4, stateMap.size()); @@ -287,10 +287,10 @@ public void testGetConnectionStates() throws IOException { public void testGetNodeIdentifiers() throws IOException { // Add a disconnected node coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(1), DisconnectionCode.LACK_OF_HEARTBEAT)); - coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(2), NodeConnectionState.DISCONNECTING, Collections.emptySet())); - coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(3), NodeConnectionState.CONNECTING, Collections.emptySet())); - coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(4), NodeConnectionState.CONNECTED, Collections.emptySet())); - coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(5), NodeConnectionState.CONNECTED, Collections.emptySet())); + coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(2), NodeConnectionState.DISCONNECTING)); + coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(3), NodeConnectionState.CONNECTING)); + coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(4), NodeConnectionState.CONNECTED)); + coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(5), NodeConnectionState.CONNECTED)); final Set connectedIds = coordinator.getNodeIdentifiers(NodeConnectionState.CONNECTED); assertEquals(2, connectedIds.size()); @@ -315,8 +315,8 @@ public void testGetNodeIdentifiers() throws IOException { public void testRequestNodeDisconnect() throws InterruptedException { // Add a connected node final NodeIdentifier nodeId1 = createNodeId(1); - coordinator.updateNodeStatus(new NodeConnectionStatus(nodeId1, NodeConnectionState.CONNECTED, Collections.emptySet())); - coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(2), NodeConnectionState.CONNECTED, Collections.emptySet())); + coordinator.updateNodeStatus(new NodeConnectionStatus(nodeId1, NodeConnectionState.CONNECTED)); + coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(2), NodeConnectionState.CONNECTED)); // wait for the status change message and clear it while (nodeStatuses.isEmpty()) { @@ -341,8 +341,8 @@ public void testCannotDisconnectLastNode() throws InterruptedException { // Add a connected node final NodeIdentifier nodeId1 = createNodeId(1); final NodeIdentifier nodeId2 = createNodeId(2); - coordinator.updateNodeStatus(new NodeConnectionStatus(nodeId1, NodeConnectionState.CONNECTED, Collections.emptySet())); - coordinator.updateNodeStatus(new NodeConnectionStatus(nodeId2, NodeConnectionState.CONNECTED, Collections.emptySet())); + coordinator.updateNodeStatus(new NodeConnectionStatus(nodeId1, NodeConnectionState.CONNECTED)); + coordinator.updateNodeStatus(new NodeConnectionStatus(nodeId2, NodeConnectionState.CONNECTED)); // wait for the status change message and clear it while (nodeStatuses.isEmpty()) { @@ -370,8 +370,8 @@ public void testUpdateNodeStatusOutOfOrder() throws InterruptedException { final NodeIdentifier nodeId1 = createNodeId(1); final NodeIdentifier nodeId2 = createNodeId(2); - coordinator.updateNodeStatus(new NodeConnectionStatus(nodeId1, NodeConnectionState.CONNECTED, Collections.emptySet())); - coordinator.updateNodeStatus(new NodeConnectionStatus(nodeId2, NodeConnectionState.CONNECTED, Collections.emptySet())); + coordinator.updateNodeStatus(new NodeConnectionStatus(nodeId1, NodeConnectionState.CONNECTED)); + coordinator.updateNodeStatus(new NodeConnectionStatus(nodeId2, NodeConnectionState.CONNECTED)); // wait for the status change message and clear it while (nodeStatuses.size() < 2) { @@ -380,7 +380,7 @@ public void testUpdateNodeStatusOutOfOrder() throws InterruptedException { nodeStatuses.clear(); final NodeConnectionStatus oldStatus = new NodeConnectionStatus(-1L, nodeId1, NodeConnectionState.DISCONNECTED, - DisconnectionCode.BLOCKED_BY_FIREWALL, null, 0L, null); + DisconnectionCode.BLOCKED_BY_FIREWALL, null, 0L); final NodeStatusChangeMessage msg = new NodeStatusChangeMessage(); msg.setNodeId(nodeId1); msg.setNodeConnectionStatus(oldStatus); @@ -391,61 +391,6 @@ public void testUpdateNodeStatusOutOfOrder() throws InterruptedException { assertTrue(nodeStatuses.isEmpty()); } - @Test(timeout = 5000) - public void testUpdateNodeRoles() throws InterruptedException { - // Add a connected node - final NodeIdentifier nodeId1 = createNodeId(1); - final NodeIdentifier nodeId2 = createNodeId(2); - - coordinator.updateNodeStatus(new NodeConnectionStatus(nodeId1, NodeConnectionState.CONNECTED, Collections.emptySet())); - // wait for the status change message and clear it - while (nodeStatuses.isEmpty()) { - Thread.sleep(10L); - } - nodeStatuses.clear(); - - coordinator.updateNodeStatus(new NodeConnectionStatus(nodeId2, NodeConnectionState.CONNECTED, Collections.emptySet())); - // wait for the status change message and clear it - while (nodeStatuses.isEmpty()) { - Thread.sleep(10L); - } - nodeStatuses.clear(); - - // Update role of node 1 to primary node - coordinator.updateNodeRoles(nodeId1, Collections.singleton(ClusterRoles.PRIMARY_NODE)); - - // wait for the status change message - while (nodeStatuses.isEmpty()) { - Thread.sleep(10L); - } - // verify the message - final NodeConnectionStatus status = nodeStatuses.get(0); - assertNotNull(status); - assertEquals(nodeId1, status.getNodeIdentifier()); - assertEquals(NodeConnectionState.CONNECTED, status.getState()); - assertEquals(Collections.singleton(ClusterRoles.PRIMARY_NODE), status.getRoles()); - nodeStatuses.clear(); - - // Update role of node 2 to primary node. This should trigger 2 status changes - - // node 1 should lose primary role & node 2 should gain it - coordinator.updateNodeRoles(nodeId2, Collections.singleton(ClusterRoles.PRIMARY_NODE)); - - // wait for the status change message - while (nodeStatuses.size() < 2) { - Thread.sleep(10L); - } - - final NodeConnectionStatus status1 = nodeStatuses.get(0); - final NodeConnectionStatus status2 = nodeStatuses.get(1); - final NodeConnectionStatus id1Msg = (status1.getNodeIdentifier().equals(nodeId1)) ? status1 : status2; - final NodeConnectionStatus id2Msg = (status1.getNodeIdentifier().equals(nodeId2)) ? status1 : status2; - - assertNotSame(id1Msg, id2Msg); - - assertTrue(id1Msg.getRoles().isEmpty()); - assertEquals(Collections.singleton(ClusterRoles.PRIMARY_NODE), id2Msg.getRoles()); - } - @Test public void testProposedIdentifierResolvedIfConflict() { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java index 59a13b28fcd2..4d7e2110b823 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java @@ -121,11 +121,11 @@ public Node createNode() { public Node waitForClusterCoordinator(final long time, final TimeUnit timeUnit) { return ClusterUtils.waitUntilNonNull(time, timeUnit, - () -> getNodes().stream().filter(node -> node.getRoles().contains(ClusterRoles.CLUSTER_COORDINATOR)).findFirst().orElse(null)); + () -> getNodes().stream().filter(node -> node.hasRole(ClusterRoles.CLUSTER_COORDINATOR)).findFirst().orElse(null)); } public Node waitForPrimaryNode(final long time, final TimeUnit timeUnit) { return ClusterUtils.waitUntilNonNull(time, timeUnit, - () -> getNodes().stream().filter(node -> node.getRoles().contains(ClusterRoles.PRIMARY_NODE)).findFirst().orElse(null)); + () -> getNodes().stream().filter(node -> node.hasRole(ClusterRoles.PRIMARY_NODE)).findFirst().orElse(null)); } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/ClusterConnectionIT.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/ClusterConnectionIT.java index 699d692cfef8..bd4cc9620ccd 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/ClusterConnectionIT.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/ClusterConnectionIT.java @@ -225,7 +225,6 @@ public void testHeartbeatsMonitored() throws IOException { final Node nodeToSuspend = firstNode; final Node otherNode = secondNode; - System.out.println("\n\n\nSuspending heartbeats on node. Roles = " + nodeToSuspend.getRoles() + "\n\n\n"); nodeToSuspend.suspendHeartbeating(); // Heartbeat interval in nifi.properties is set to 1 sec. This means that the node should be kicked out diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Node.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Node.java index ec57992b3aa7..2996442c465b 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Node.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Node.java @@ -18,11 +18,11 @@ package org.apache.nifi.cluster.integration; import java.io.IOException; +import java.net.InetSocketAddress; import java.net.ServerSocket; import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.Set; import java.util.UUID; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -201,11 +201,6 @@ private static int createPort() { } } - public Set getRoles() { - final NodeConnectionStatus status = getConnectionStatus(); - return status == null ? Collections.emptySet() : status.getRoles(); - } - public NodeConnectionStatus getConnectionStatus() { return clusterCoordinator.getConnectionStatus(nodeId); } @@ -284,8 +279,22 @@ public void waitUntilConnected(final long time, final TimeUnit timeUnit) { ClusterUtils.waitUntilConditionMet(time, timeUnit, () -> isConnected()); } + private String getClusterAddress() { + final InetSocketAddress address = nodeProperties.getClusterNodeProtocolAddress(); + return address.getHostName() + ":" + address.getPort(); + } + + public boolean hasRole(final String roleName) { + final String leaderAddress = electionManager.getLeader(roleName); + if (leaderAddress == null) { + return false; + } + + return leaderAddress.equals(getClusterAddress()); + } + public void waitUntilElectedForRole(final String roleName, final long time, final TimeUnit timeUnit) { - ClusterUtils.waitUntilConditionMet(time, timeUnit, () -> getRoles().contains(roleName)); + ClusterUtils.waitUntilConditionMet(time, timeUnit, () -> hasRole(roleName)); } // Assertions diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java index 6599bccdfabb..4fb5aef5e858 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java @@ -3326,25 +3326,19 @@ public synchronized void onLeaderRelinquish() { // call start() when we become the leader, and this will ensure that initialization is handled. The heartbeat monitor // then will check the zookeeper znode to check if it is the cluster coordinator before kicking any nodes out of the // cluster. - - if (clusterCoordinator != null) { - clusterCoordinator.removeRole(ClusterRoles.CLUSTER_COORDINATOR); - } } @Override public synchronized void onLeaderElection() { LOG.info("This node elected Active Cluster Coordinator"); heartbeatMonitor.start(); // ensure heartbeat monitor is started - - if (clusterCoordinator != null) { - clusterCoordinator.addRole(ClusterRoles.CLUSTER_COORDINATOR); - } } }, participantId); } private void registerForPrimaryNode() { + final String participantId = heartbeatMonitor.getHeartbeatAddress(); + leaderElectionManager.register(ClusterRoles.PRIMARY_NODE, new LeaderElectionStateChangeListener() { @Override public void onLeaderElection() { @@ -3355,7 +3349,7 @@ public void onLeaderElection() { public void onLeaderRelinquish() { setPrimary(false); } - }); + }, participantId); } /** @@ -3930,15 +3924,7 @@ HeartbeatMessage createHeartbeatMessage() { return null; } - final Set roles = new HashSet<>(); - if (bean.isPrimary()) { - roles.add(ClusterRoles.PRIMARY_NODE); - } - if (clusterCoordinator.isActiveClusterCoordinator()) { - roles.add(ClusterRoles.CLUSTER_COORDINATOR); - } - - final Heartbeat heartbeat = new Heartbeat(nodeId, roles, connectionStatus, hbPayload.marshal()); + final Heartbeat heartbeat = new Heartbeat(nodeId, connectionStatus, hbPayload.marshal()); final HeartbeatMessage message = new HeartbeatMessage(); message.setHeartbeat(heartbeat); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java index 091e59ca096c..42f239c0b3cf 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java @@ -35,7 +35,6 @@ import java.util.Date; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.UUID; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -853,9 +852,7 @@ private void loadFromConnectionResponse(final ConnectionResponse response) throw // mark the node as clustered controller.setClustered(true, response.getInstanceId()); - final NodeConnectionStatus status = clusterCoordinator.getConnectionStatus(nodeId); - final Set roles = status == null ? Collections.emptySet() : status.getRoles(); - controller.setConnectionStatus(new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTED, roles)); + controller.setConnectionStatus(new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTED)); // start the processors as indicated by the dataflow controller.onFlowInitialized(autoResumeState); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java index 34890adc71e0..3dfffce6e8db 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java @@ -42,6 +42,7 @@ import org.apache.nifi.cluster.coordination.ClusterCoordinator; import org.apache.nifi.cluster.coordination.heartbeat.HeartbeatMonitor; import org.apache.nifi.cluster.coordination.heartbeat.NodeHeartbeat; +import org.apache.nifi.cluster.coordination.node.ClusterRoles; import org.apache.nifi.cluster.coordination.node.DisconnectionCode; import org.apache.nifi.cluster.coordination.node.NodeConnectionState; import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; @@ -65,6 +66,7 @@ import org.apache.nifi.controller.Snippet; import org.apache.nifi.controller.Template; import org.apache.nifi.controller.label.Label; +import org.apache.nifi.controller.leader.election.LeaderElectionManager; import org.apache.nifi.controller.repository.claim.ContentDirection; import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.controller.service.ControllerServiceReference; @@ -261,6 +263,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { private AccessPolicyDAO accessPolicyDAO; private ClusterCoordinator clusterCoordinator; private HeartbeatMonitor heartbeatMonitor; + private LeaderElectionManager leaderElectionManager; // administrative services private AuditService auditService; @@ -3116,19 +3119,10 @@ public ClusterDTO getCluster() { clusterDto.setGenerated(new Date()); // create node dtos - final Collection nodeDtos = new ArrayList<>(); + final List nodeDtos = clusterCoordinator.getNodeIdentifiers().stream() + .map(nodeId -> getNode(nodeId)) + .collect(Collectors.toList()); clusterDto.setNodes(nodeDtos); - for (final NodeIdentifier nodeId : clusterCoordinator.getNodeIdentifiers()) { - final NodeConnectionStatus status = clusterCoordinator.getConnectionStatus(nodeId); - if (status == null) { - continue; - } - - final List events = clusterCoordinator.getNodeEvents(nodeId); - final Set nodeRoles = clusterCoordinator.getConnectionStatus(nodeId).getRoles(); - final NodeHeartbeat heartbeat = heartbeatMonitor.getLatestHeartbeat(nodeId); - nodeDtos.add(dtoFactory.createNodeDTO(nodeId, status, heartbeat, events, nodeRoles)); - } return clusterDto; } @@ -3142,11 +3136,29 @@ public NodeDTO getNode(final String nodeId) { private NodeDTO getNode(final NodeIdentifier nodeId) { final NodeConnectionStatus nodeStatus = clusterCoordinator.getConnectionStatus(nodeId); final List events = clusterCoordinator.getNodeEvents(nodeId); - final Set roles = clusterCoordinator.getConnectionStatus(nodeId).getRoles(); + final Set roles = getRoles(nodeId); final NodeHeartbeat heartbeat = heartbeatMonitor.getLatestHeartbeat(nodeId); return dtoFactory.createNodeDTO(nodeId, nodeStatus, heartbeat, events, roles); } + private Set getRoles(final NodeIdentifier nodeId) { + final Set roles = new HashSet<>(); + final String nodeAddress = nodeId.getSocketAddress() + ":" + nodeId.getSocketPort(); + + for (final String roleName : ClusterRoles.getAllRoles()) { + final String leader = leaderElectionManager.getLeader(roleName); + if (leader == null) { + continue; + } + + if (leader.equals(nodeAddress)) { + roles.add(roleName); + } + } + + return roles; + } + @Override public void deleteNode(final String nodeId) { final NiFiUser user = NiFiUserUtils.getNiFiUser(); @@ -3290,4 +3302,8 @@ public void setHeartbeatMonitor(final HeartbeatMonitor heartbeatMonitor) { public void setBulletinRepository(final BulletinRepository bulletinRepository) { this.bulletinRepository = bulletinRepository; } + + public void setLeaderElectionManager(final LeaderElectionManager leaderElectionManager) { + this.leaderElectionManager = leaderElectionManager; + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml index 28dbb6240755..614043aa1b7c 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml @@ -163,6 +163,7 @@ + From 7ba4d06b5d455fb2c17f68f201b7312bb8bb17d3 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Mon, 15 Aug 2016 16:35:56 -0400 Subject: [PATCH 3/4] NIFI-2566: Updated Heartbeats so that if a node is out-of-sync with cluster topology, cluster coordinator will provide updated information back to the nodes --- .../coordination/ClusterCoordinator.java | 18 + .../node/NodeConnectionStatus.java | 4 +- .../protocol/AbstractNodeProtocolSender.java | 24 +- .../cluster/protocol}/HeartbeatPayload.java | 14 +- .../cluster/protocol/NodeProtocolSender.java | 5 +- .../impl/NodeProtocolSenderListener.java | 5 +- .../protocol/jaxb/message/ObjectFactory.java | 4 + .../message/HeartbeatResponseMessage.java | 45 +++ .../protocol/message/ProtocolMessage.java | 1 + .../jaxb/message/TestJaxbProtocolUtils.java | 28 ++ .../ClusterProtocolHeartbeatMonitor.java | 49 ++- .../heartbeat/StandardNodeHeartbeat.java | 3 +- .../node/NodeClusterCoordinator.java | 95 +++-- .../TestAbstractHeartbeatMonitor.java | 10 + .../nifi/cluster/integration/Cluster.java | 13 + .../integration/ClusterConnectionIT.java | 327 +++++++++--------- .../nifi/controller/FlowController.java | 7 +- .../cluster/ClusterProtocolHeartbeater.java | 39 ++- .../nifi/cluster/HeartbeatPayloadTest.java | 1 + .../org/apache/nifi/nar/NarCloseable.java | 29 ++ 20 files changed, 503 insertions(+), 218 deletions(-) rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-framework-core/src/main/java/org/apache/nifi/cluster => nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol}/HeartbeatPayload.java (90%) create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/HeartbeatResponseMessage.java diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/ClusterCoordinator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/ClusterCoordinator.java index ff8d06ecafb7..49c6142814cb 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/ClusterCoordinator.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/ClusterCoordinator.java @@ -113,6 +113,13 @@ public interface ClusterCoordinator { */ Map> getConnectionStates(); + /** + * Returns a List of the NodeConnectionStatus for each node in the cluster + * + * @return a List of the NodeConnectionStatus for each node in the cluster + */ + List getConnectionStatuses(); + /** * Checks if the given hostname is blocked by the configured firewall, returning * true if the node is blocked, false if the node is @@ -188,6 +195,17 @@ public interface ClusterCoordinator { */ void resetNodeStatuses(Map statusMap); + /** + * Resets the status of the node to be in accordance with the given NodeConnectionStatus if and only if the + * currently held status for this node has an Update ID equal to the given qualifyingUpdateId + * + * @param connectionStatus the new status of the node + * @param qualifyingUpdateId the Update ID to compare the current ID with. If the current ID for the node described by the provided + * NodeConnectionStatus is not equal to this value, the value will not be updated + * @return true if the node status was updated, false if the qualifyingUpdateId is out of date. + */ + boolean resetNodeStatus(NodeConnectionStatus connectionStatus, long qualifyingUpdateId); + /** * Notifies the Cluster Coordinator of the Node Identifier that the coordinator is currently running on * diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/NodeConnectionStatus.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/NodeConnectionStatus.java index c3862ad1574d..34bd1279e3e7 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/NodeConnectionStatus.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/NodeConnectionStatus.java @@ -53,7 +53,7 @@ public NodeConnectionStatus(final NodeIdentifier nodeId, final DisconnectionCode } public NodeConnectionStatus(final NodeIdentifier nodeId, final NodeConnectionState state, final DisconnectionCode disconnectionCode) { - this(nodeId, state, disconnectionCode, disconnectionCode.toString(), null); + this(nodeId, state, disconnectionCode, disconnectionCode == null ? null : disconnectionCode.toString(), null); } public NodeConnectionStatus(final NodeConnectionStatus status) { @@ -109,7 +109,7 @@ public Long getConnectionRequestTime() { public String toString() { final StringBuilder sb = new StringBuilder(); final NodeConnectionState state = getState(); - sb.append("NodeConnectionStatus[state=").append(state); + sb.append("NodeConnectionStatus[nodeId=").append(nodeId).append(", state=").append(state); if (state == NodeConnectionState.DISCONNECTED || state == NodeConnectionState.DISCONNECTING) { sb.append(", Disconnect Code=").append(getDisconnectCode()).append(", Disconnect Reason=").append(getDisconnectReason()); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/AbstractNodeProtocolSender.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/AbstractNodeProtocolSender.java index 4131dc5f1ec6..22d6ebc65171 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/AbstractNodeProtocolSender.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/AbstractNodeProtocolSender.java @@ -24,6 +24,7 @@ import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage; import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage; import org.apache.nifi.cluster.protocol.message.HeartbeatMessage; +import org.apache.nifi.cluster.protocol.message.HeartbeatResponseMessage; import org.apache.nifi.cluster.protocol.message.ProtocolMessage; import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType; import org.apache.nifi.io.socket.SocketConfiguration; @@ -74,7 +75,7 @@ public ConnectionResponseMessage requestConnection(final ConnectionRequestMessag } @Override - public void heartbeat(final HeartbeatMessage msg, final String address) throws ProtocolException { + public HeartbeatResponseMessage heartbeat(final HeartbeatMessage msg, final String address) throws ProtocolException { final String hostname; final int port; try { @@ -85,7 +86,12 @@ public void heartbeat(final HeartbeatMessage msg, final String address) throws P throw new IllegalArgumentException("Cannot send heartbeat to address [" + address + "]. Address must be in : format"); } - sendProtocolMessage(msg, hostname, port); + final ProtocolMessage responseMessage = sendProtocolMessage(msg, hostname, port); + if (MessageType.HEARTBEAT_RESPONSE == responseMessage.getType()) { + return (HeartbeatResponseMessage) responseMessage; + } + + throw new ProtocolException("Expected message type '" + MessageType.HEARTBEAT_RESPONSE + "' but found '" + responseMessage.getType() + "'"); } @@ -108,7 +114,7 @@ public SocketConfiguration getSocketConfiguration() { return socketConfiguration; } - private void sendProtocolMessage(final ProtocolMessage msg, final String hostname, final int port) { + private ProtocolMessage sendProtocolMessage(final ProtocolMessage msg, final String hostname, final int port) { Socket socket = null; try { try { @@ -124,6 +130,18 @@ private void sendProtocolMessage(final ProtocolMessage msg, final String hostnam } catch (final IOException ioe) { throw new ProtocolException("Failed marshalling '" + msg.getType() + "' protocol message due to: " + ioe, ioe); } + + final ProtocolMessage response; + try { + // unmarshall response and return + final ProtocolMessageUnmarshaller unmarshaller = protocolContext.createUnmarshaller(); + response = unmarshaller.unmarshal(socket.getInputStream()); + } catch (final IOException ioe) { + throw new ProtocolException("Failed unmarshalling '" + MessageType.CONNECTION_RESPONSE + "' protocol message from " + + socket.getRemoteSocketAddress() + " due to: " + ioe, ioe); + } + + return response; } finally { SocketUtils.closeQuietly(socket); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/cluster/HeartbeatPayload.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/HeartbeatPayload.java similarity index 90% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/cluster/HeartbeatPayload.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/HeartbeatPayload.java index 1146a39547b7..8363a2013435 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/cluster/HeartbeatPayload.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/HeartbeatPayload.java @@ -14,12 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.cluster; +package org.apache.nifi.cluster.protocol; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.InputStream; import java.io.OutputStream; +import java.util.List; import javax.xml.bind.JAXBContext; import javax.xml.bind.JAXBException; @@ -27,7 +28,7 @@ import javax.xml.bind.Unmarshaller; import javax.xml.bind.annotation.XmlRootElement; -import org.apache.nifi.cluster.protocol.ProtocolException; +import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; /** * The payload of the heartbeat. The payload contains status to inform the cluster manager the current workload of this node. @@ -50,6 +51,7 @@ public class HeartbeatPayload { private long totalFlowFileCount; private long totalFlowFileBytes; private long systemStartTime; + private List clusterStatus; public int getActiveThreadCount() { return activeThreadCount; @@ -83,6 +85,14 @@ public void setSystemStartTime(final long systemStartTime) { this.systemStartTime = systemStartTime; } + public List getClusterStatus() { + return clusterStatus; + } + + public void setClusterStatus(final List clusterStatus) { + this.clusterStatus = clusterStatus; + } + public byte[] marshal() throws ProtocolException { final ByteArrayOutputStream payloadBytes = new ByteArrayOutputStream(); marshal(this, payloadBytes); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeProtocolSender.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeProtocolSender.java index 432a03d24da6..fcf519540ed7 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeProtocolSender.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeProtocolSender.java @@ -19,6 +19,7 @@ import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage; import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage; import org.apache.nifi.cluster.protocol.message.HeartbeatMessage; +import org.apache.nifi.cluster.protocol.message.HeartbeatResponseMessage; /** * An interface for sending protocol messages from a node to the cluster @@ -44,6 +45,8 @@ public interface NodeProtocolSender { * @param msg the heartbeat message to send * @param address the address of the Cluster Coordinator in <hostname>:<port> format * @throws ProtocolException if unable to send the heartbeat + * + * @return the response from the Cluster Coordinator */ - void heartbeat(HeartbeatMessage msg, String address) throws ProtocolException; + HeartbeatResponseMessage heartbeat(HeartbeatMessage msg, String address) throws ProtocolException; } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderListener.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderListener.java index 0fd2517aa1d5..1b0aeeab3247 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderListener.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderListener.java @@ -27,6 +27,7 @@ import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage; import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage; import org.apache.nifi.cluster.protocol.message.HeartbeatMessage; +import org.apache.nifi.cluster.protocol.message.HeartbeatResponseMessage; import org.apache.nifi.reporting.BulletinRepository; public class NodeProtocolSenderListener implements NodeProtocolSender, ProtocolListener { @@ -92,7 +93,7 @@ public void setBulletinRepository(final BulletinRepository bulletinRepository) { } @Override - public void heartbeat(HeartbeatMessage msg, String address) throws ProtocolException { - sender.heartbeat(msg, address); + public HeartbeatResponseMessage heartbeat(final HeartbeatMessage msg, final String address) throws ProtocolException { + return sender.heartbeat(msg, address); } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ObjectFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ObjectFactory.java index da13b02375bf..afa87b9f8a05 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ObjectFactory.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ObjectFactory.java @@ -24,6 +24,7 @@ import org.apache.nifi.cluster.protocol.message.FlowRequestMessage; import org.apache.nifi.cluster.protocol.message.FlowResponseMessage; import org.apache.nifi.cluster.protocol.message.HeartbeatMessage; +import org.apache.nifi.cluster.protocol.message.HeartbeatResponseMessage; import org.apache.nifi.cluster.protocol.message.MulticastProtocolMessage; import org.apache.nifi.cluster.protocol.message.NodeConnectionStatusRequestMessage; import org.apache.nifi.cluster.protocol.message.NodeConnectionStatusResponseMessage; @@ -97,4 +98,7 @@ public NodeConnectionStatusResponseMessage createNodeConnectionStatusResponsetMe return new NodeConnectionStatusResponseMessage(); } + public HeartbeatResponseMessage createHeartbeatResponse() { + return new HeartbeatResponseMessage(); + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/HeartbeatResponseMessage.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/HeartbeatResponseMessage.java new file mode 100644 index 000000000000..cbb8b488d40f --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/HeartbeatResponseMessage.java @@ -0,0 +1,45 @@ +/* + * 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.nifi.cluster.protocol.message; + +import java.util.ArrayList; +import java.util.List; + +import javax.xml.bind.annotation.XmlRootElement; + +import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; + +@XmlRootElement(name = "heartbeatResponse") +public class HeartbeatResponseMessage extends ProtocolMessage { + + private List updatedNodeStatuses = new ArrayList<>(); + + + @Override + public MessageType getType() { + return MessageType.HEARTBEAT_RESPONSE; + } + + public List getUpdatedNodeStatuses() { + return updatedNodeStatuses; + } + + public void setUpdatedNodeStatuses(final List nodeStatuses) { + this.updatedNodeStatuses = new ArrayList<>(nodeStatuses); + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ProtocolMessage.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ProtocolMessage.java index 2e74689252b9..1d0d1159e4fe 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ProtocolMessage.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ProtocolMessage.java @@ -32,6 +32,7 @@ public static enum MessageType { RECONNECTION_RESPONSE, SERVICE_BROADCAST, HEARTBEAT, + HEARTBEAT_RESPONSE, NODE_CONNECTION_STATUS_REQUEST, NODE_CONNECTION_STATUS_RESPONSE, NODE_STATUS_CHANGE; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/jaxb/message/TestJaxbProtocolUtils.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/jaxb/message/TestJaxbProtocolUtils.java index 21636a4bcc86..4fa53e888908 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/jaxb/message/TestJaxbProtocolUtils.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/jaxb/message/TestJaxbProtocolUtils.java @@ -32,9 +32,12 @@ import org.apache.nifi.cluster.protocol.ComponentRevision; import org.apache.nifi.cluster.protocol.ConnectionResponse; import org.apache.nifi.cluster.protocol.DataFlow; +import org.apache.nifi.cluster.protocol.Heartbeat; +import org.apache.nifi.cluster.protocol.HeartbeatPayload; import org.apache.nifi.cluster.protocol.NodeIdentifier; import org.apache.nifi.cluster.protocol.StandardDataFlow; import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage; +import org.apache.nifi.cluster.protocol.message.HeartbeatMessage; import org.apache.nifi.cluster.protocol.message.NodeConnectionStatusRequestMessage; import org.apache.nifi.cluster.protocol.message.NodeConnectionStatusResponseMessage; import org.apache.nifi.web.Revision; @@ -96,4 +99,29 @@ public void testRoundTripConnectionStatusResponse() throws JAXBException { final NodeConnectionStatus unmarshalledStatus = unmarshalledMsg.getNodeConnectionStatus(); assertEquals(nodeStatus, unmarshalledStatus); } + + @Test + public void testRoundTripHeartbeat() throws JAXBException { + final NodeIdentifier nodeId = new NodeIdentifier("id", "localhost", 8000, "localhost", 8001, "localhost", 8002, 8003, true); + final NodeConnectionStatus nodeStatus = new NodeConnectionStatus(nodeId, DisconnectionCode.NOT_YET_CONNECTED); + + final HeartbeatPayload payload = new HeartbeatPayload(); + payload.setActiveThreadCount(1); + payload.setSystemStartTime(System.currentTimeMillis()); + payload.setTotalFlowFileBytes(83L); + payload.setTotalFlowFileCount(4); + + final List clusterStatus = Collections.singletonList(nodeStatus); + payload.setClusterStatus(clusterStatus); + + final Heartbeat heartbeat = new Heartbeat(nodeId, nodeStatus, payload.marshal()); + + final HeartbeatMessage msg = new HeartbeatMessage(); + msg.setHeartbeat(heartbeat); + + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + JaxbProtocolUtils.JAXB_CONTEXT.createMarshaller().marshal(msg, baos); + final Object unmarshalled = JaxbProtocolUtils.JAXB_CONTEXT.createUnmarshaller().unmarshal(new ByteArrayInputStream(baos.toByteArray())); + assertTrue(unmarshalled instanceof HeartbeatMessage); + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/ClusterProtocolHeartbeatMonitor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/ClusterProtocolHeartbeatMonitor.java index 3afd6a63b82d..b955bd008661 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/ClusterProtocolHeartbeatMonitor.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/ClusterProtocolHeartbeatMonitor.java @@ -17,24 +17,31 @@ package org.apache.nifi.cluster.coordination.heartbeat; +import java.util.ArrayList; import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.function.Function; +import java.util.stream.Collectors; import javax.xml.bind.JAXBContext; import javax.xml.bind.Unmarshaller; -import org.apache.nifi.cluster.HeartbeatPayload; import org.apache.nifi.cluster.coordination.ClusterCoordinator; +import org.apache.nifi.cluster.coordination.node.NodeConnectionState; import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; import org.apache.nifi.cluster.protocol.Heartbeat; +import org.apache.nifi.cluster.protocol.HeartbeatPayload; import org.apache.nifi.cluster.protocol.NodeIdentifier; import org.apache.nifi.cluster.protocol.ProtocolException; import org.apache.nifi.cluster.protocol.ProtocolHandler; import org.apache.nifi.cluster.protocol.ProtocolListener; import org.apache.nifi.cluster.protocol.message.HeartbeatMessage; +import org.apache.nifi.cluster.protocol.message.HeartbeatResponseMessage; import org.apache.nifi.cluster.protocol.message.ProtocolMessage; import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType; import org.apache.nifi.util.NiFiProperties; @@ -148,7 +155,45 @@ public ProtocolMessage handle(final ProtocolMessage msg) throws ProtocolExceptio heartbeatMessages.put(heartbeat.getNodeIdentifier(), nodeHeartbeat); logger.debug("Received new heartbeat from {}", nodeId); - return null; + // Formulate a List of differences between our view of the cluster topology and the node's view + // and send that back to the node so that it is in-sync with us + final List nodeStatusList = payload.getClusterStatus(); + final List updatedStatuses = getUpdatedStatuses(nodeStatusList); + + final HeartbeatResponseMessage responseMessage = new HeartbeatResponseMessage(); + responseMessage.setUpdatedNodeStatuses(updatedStatuses); + return responseMessage; + } + + + private List getUpdatedStatuses(final List nodeStatusList) { + // Map node's statuses by NodeIdentifier for quick & easy lookup + final Map nodeStatusMap = nodeStatusList.stream() + .collect(Collectors.toMap(status -> status.getNodeIdentifier(), Function.identity())); + + // Check if our connection status is the same for each Node Identifier and if not, add our version of the status + // to a List of updated statuses. + final List currentStatuses = clusterCoordinator.getConnectionStatuses(); + final List updatedStatuses = new ArrayList<>(); + for (final NodeConnectionStatus currentStatus : currentStatuses) { + final NodeConnectionStatus nodeStatus = nodeStatusMap.get(currentStatus.getNodeIdentifier()); + if (!currentStatus.equals(nodeStatus)) { + updatedStatuses.add(currentStatus); + } + } + + // If the node has any statuses that we do not have, add a REMOVED status to the update list + final Set nodeIds = currentStatuses.stream().map(status -> status.getNodeIdentifier()).collect(Collectors.toSet()); + for (final NodeConnectionStatus nodeStatus : nodeStatusList) { + if (!nodeIds.contains(nodeStatus.getNodeIdentifier())) { + updatedStatuses.add(new NodeConnectionStatus(nodeStatus.getNodeIdentifier(), NodeConnectionState.REMOVED, null)); + } + } + + logger.debug("\n\nCalculated diff between current cluster status and node cluster status as follows:\nNode: {}\nSelf: {}\nDifference: {}\n\n", + nodeStatusList, currentStatuses, updatedStatuses); + + return updatedStatuses; } @Override diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/StandardNodeHeartbeat.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/StandardNodeHeartbeat.java index 8fec4cf0cf2c..a63db4488c9a 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/StandardNodeHeartbeat.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/StandardNodeHeartbeat.java @@ -17,9 +17,9 @@ package org.apache.nifi.cluster.coordination.heartbeat; -import org.apache.nifi.cluster.HeartbeatPayload; import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; import org.apache.nifi.cluster.protocol.Heartbeat; +import org.apache.nifi.cluster.protocol.HeartbeatPayload; import org.apache.nifi.cluster.protocol.NodeIdentifier; import org.apache.nifi.cluster.protocol.message.HeartbeatMessage; @@ -74,7 +74,6 @@ public int getActiveThreadCount() { return activeThreadCount; } - @Override public long getSystemStartTime() { return systemStartTime; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java index faea59fd6960..8e580dbc7af8 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java @@ -28,6 +28,7 @@ import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicLong; import java.util.function.Supplier; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -83,6 +84,7 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl private final ClusterNodeFirewall firewall; private final RevisionManager revisionManager; private final LeaderElectionManager leaderElectionManager; + private final AtomicLong latestUpdateId = new AtomicLong(-1); private volatile FlowService flowService; private volatile boolean connected; @@ -167,12 +169,29 @@ public void resetNodeStatuses(final Map st final NodeIdentifier nodeId = entry.getKey(); final NodeConnectionStatus proposedStatus = entry.getValue(); - boolean updated = false; - while (!updated) { - final NodeConnectionStatus currentStatus = nodeStatuses.get(nodeId); - updated = replaceNodeStatus(nodeId, currentStatus, proposedStatus); + if (proposedStatus.getState() == NodeConnectionState.REMOVED) { + nodeStatuses.remove(nodeId); + } else { + nodeStatuses.put(nodeId, proposedStatus); + } + } + } + + @Override + public boolean resetNodeStatus(final NodeConnectionStatus connectionStatus, final long qualifyingUpdateId) { + final NodeIdentifier nodeId = connectionStatus.getNodeIdentifier(); + final NodeConnectionStatus currentStatus = getConnectionStatus(nodeId); + + if (currentStatus == null) { + return replaceNodeStatus(nodeId, null, connectionStatus); + } else { + if (currentStatus.getUpdateIdentifier() == qualifyingUpdateId) { + return replaceNodeStatus(nodeId, currentStatus, connectionStatus); } } + + // The update identifier is not the same. We will not replace the value + return false; } /** @@ -192,11 +211,19 @@ private boolean replaceNodeStatus(final NodeIdentifier nodeId, final NodeConnect } if (currentStatus == null) { - final NodeConnectionStatus existingValue = nodeStatuses.putIfAbsent(nodeId, newStatus); - return existingValue == null; + if (newStatus.getState() == NodeConnectionState.REMOVED) { + return nodeStatuses.remove(nodeId, currentStatus); + } else { + final NodeConnectionStatus existingValue = nodeStatuses.putIfAbsent(nodeId, newStatus); + return existingValue == null; + } } - return nodeStatuses.replace(nodeId, currentStatus, newStatus); + if (newStatus.getState() == NodeConnectionState.REMOVED) { + return nodeStatuses.remove(nodeId, currentStatus); + } else { + return nodeStatuses.replace(nodeId, currentStatus, newStatus); + } } @Override @@ -309,6 +336,10 @@ private NodeConnectionState getConnectionState(final NodeIdentifier nodeId) { return status == null ? null : status.getState(); } + @Override + public List getConnectionStatuses() { + return new ArrayList<>(nodeStatuses.values()); + } @Override public Map> getConnectionStates() { @@ -542,6 +573,8 @@ void updateNodeStatus(final NodeConnectionStatus status, final boolean waitForCo logger.info("Status of {} changed from {} to {}", nodeId, currentStatus, status); logger.debug("State of cluster nodes is now {}", nodeStatuses); + latestUpdateId.updateAndGet(curVal -> Math.max(curVal, status.getUpdateIdentifier())); + if (currentState == null || currentState != status.getState()) { final boolean notifyAllNodes = isActiveClusterCoordinator(); if (notifyAllNodes) { @@ -556,6 +589,7 @@ void updateNodeStatus(final NodeConnectionStatus status, final boolean waitForCo } } + void notifyOthersOfNodeStatusChange(final NodeConnectionStatus updatedStatus) { notifyOthersOfNodeStatusChange(updatedStatus, isActiveClusterCoordinator(), true); } @@ -643,7 +677,7 @@ public void run() { } request.setDataFlow(new StandardDataFlow(flowService.createDataFlow())); - request.setNodeConnectionStatuses(new ArrayList<>(nodeStatuses.values())); + request.setNodeConnectionStatuses(getConnectionStatuses()); request.setComponentRevisions(revisionManager.getAllRevisions().stream().map(rev -> ComponentRevision.fromRevision(rev)).collect(Collectors.toList())); // Issue a reconnection request to the node. @@ -719,35 +753,30 @@ private void handleNodeStatusChange(final NodeStatusChangeMessage statusChangeMe final NodeIdentifier nodeId = statusChangeMessage.getNodeId(); logger.debug("Handling request {}", statusChangeMessage); - boolean updated = false; - while (!updated) { - final NodeConnectionStatus oldStatus = nodeStatuses.get(statusChangeMessage.getNodeId()); - - // Either remove the value from the map or update the map depending on the connection state - if (statusChangeMessage.getNodeConnectionStatus().getState() == NodeConnectionState.REMOVED) { - updated = nodeStatuses.remove(nodeId, oldStatus); - } else { - updated = replaceNodeStatus(nodeId, oldStatus, updatedStatus); - } + final NodeConnectionStatus oldStatus = nodeStatuses.get(statusChangeMessage.getNodeId()); - if (updated) { - logger.info("Status of {} changed from {} to {}", statusChangeMessage.getNodeId(), oldStatus, updatedStatus); - logger.debug("State of cluster nodes is now {}", nodeStatuses); + // Either remove the value from the map or update the map depending on the connection state + if (statusChangeMessage.getNodeConnectionStatus().getState() == NodeConnectionState.REMOVED) { + nodeStatuses.remove(nodeId, oldStatus); + } else { + nodeStatuses.put(nodeId, updatedStatus); + } - final NodeConnectionStatus status = statusChangeMessage.getNodeConnectionStatus(); - final String summary = summarizeStatusChange(oldStatus, status); - if (!StringUtils.isEmpty(summary)) { - addNodeEvent(nodeId, summary); - } + logger.info("Status of {} changed from {} to {}", statusChangeMessage.getNodeId(), oldStatus, updatedStatus); + logger.debug("State of cluster nodes is now {}", nodeStatuses); - // Update our counter so that we are in-sync with the cluster on the - // most up-to-date version of the NodeConnectionStatus' Update Identifier. - // We do this so that we can accurately compare status updates that are generated - // locally against those generated from other nodes in the cluster. - NodeConnectionStatus.updateIdGenerator(updatedStatus.getUpdateIdentifier()); - } + final NodeConnectionStatus status = statusChangeMessage.getNodeConnectionStatus(); + final String summary = summarizeStatusChange(oldStatus, status); + if (!StringUtils.isEmpty(summary)) { + addNodeEvent(nodeId, summary); } + // Update our counter so that we are in-sync with the cluster on the + // most up-to-date version of the NodeConnectionStatus' Update Identifier. + // We do this so that we can accurately compare status updates that are generated + // locally against those generated from other nodes in the cluster. + NodeConnectionStatus.updateIdGenerator(updatedStatus.getUpdateIdentifier()); + if (isActiveClusterCoordinator()) { notifyOthersOfNodeStatusChange(statusChangeMessage.getNodeConnectionStatus()); } @@ -829,7 +858,7 @@ private ConnectionResponse createConnectionResponse(final ConnectionRequest requ return new ConnectionResponse(tryAgainSeconds); } - return new ConnectionResponse(resolvedNodeIdentifier, dataFlow, instanceId, new ArrayList<>(nodeStatuses.values()), + return new ConnectionResponse(resolvedNodeIdentifier, dataFlow, instanceId, getConnectionStatuses(), revisionManager.getAllRevisions().stream().map(rev -> ComponentRevision.fromRevision(rev)).collect(Collectors.toList())); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java index 06bae42868c3..46ea49b1a2fc 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java @@ -309,6 +309,16 @@ public boolean isActiveClusterCoordinator() { public NodeIdentifier getLocalNodeIdentifier() { return null; } + + @Override + public List getConnectionStatuses() { + return Collections.emptyList(); + } + + @Override + public boolean resetNodeStatus(NodeConnectionStatus connectionStatus, long qualifyingUpdateId) { + return false; + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java index 4d7e2110b823..fd54203a68d7 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java @@ -128,4 +128,17 @@ public Node waitForPrimaryNode(final long time, final TimeUnit timeUnit) { return ClusterUtils.waitUntilNonNull(time, timeUnit, () -> getNodes().stream().filter(node -> node.hasRole(ClusterRoles.PRIMARY_NODE)).findFirst().orElse(null)); } + + /** + * Waits for each node in the cluster to connect. The time given is the maximum amount of time to wait for each node to connect, not for + * the entire cluster to connect. + * + * @param time the max amount of time to wait for a node to connect + * @param timeUnit the unit of time that the given time value represents + */ + public void waitUntilAllNodesConnected(final long time, final TimeUnit timeUnit) { + for (final Node node : nodes) { + node.waitUntilConnected(time, timeUnit); + } + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/ClusterConnectionIT.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/ClusterConnectionIT.java index bd4cc9620ccd..3439263cc1b1 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/ClusterConnectionIT.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/ClusterConnectionIT.java @@ -17,224 +17,219 @@ package org.apache.nifi.cluster.integration; +import static org.junit.Assert.assertEquals; + import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; import java.util.concurrent.TimeUnit; import java.util.stream.Stream; import org.apache.nifi.cluster.coordination.node.ClusterRoles; +import org.apache.nifi.cluster.coordination.node.DisconnectionCode; import org.apache.nifi.cluster.coordination.node.NodeConnectionState; +import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.junit.After; +import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; public class ClusterConnectionIT { + private Cluster cluster; @BeforeClass public static void setup() { System.setProperty("nifi.properties.file.path", "src/test/resources/conf/nifi.properties"); } - @Test(timeout = 20000) - public void testSingleNode() throws InterruptedException { - final Cluster cluster = new Cluster(); + @Before + public void createCluster() { + cluster = new Cluster(); cluster.start(); + } - try { - final Node firstNode = cluster.createNode(); - firstNode.waitUntilConnected(10, TimeUnit.SECONDS); - - firstNode.waitUntilElectedForRole(ClusterRoles.CLUSTER_COORDINATOR, 10, TimeUnit.SECONDS); - firstNode.waitUntilElectedForRole(ClusterRoles.PRIMARY_NODE, 10, TimeUnit.SECONDS); - } finally { + @After + public void destroyCluster() { + if (cluster != null) { cluster.stop(); } } + @Test(timeout = 20000) + public void testSingleNode() throws InterruptedException { + final Node firstNode = cluster.createNode(); + firstNode.waitUntilConnected(10, TimeUnit.SECONDS); + + firstNode.waitUntilElectedForRole(ClusterRoles.CLUSTER_COORDINATOR, 10, TimeUnit.SECONDS); + firstNode.waitUntilElectedForRole(ClusterRoles.PRIMARY_NODE, 10, TimeUnit.SECONDS); + } + @Test(timeout = 60000) public void testThreeNodeCluster() throws InterruptedException { - final Cluster cluster = new Cluster(); - cluster.start(); - - try { - final Node firstNode = cluster.createNode(); - final Node secondNode = cluster.createNode(); - final Node thirdNode = cluster.createNode(); - - firstNode.waitUntilConnected(10, TimeUnit.SECONDS); - System.out.println("**** Node 1 Connected ****"); - secondNode.waitUntilConnected(10, TimeUnit.SECONDS); - System.out.println("**** Node 2 Connected ****"); - thirdNode.waitUntilConnected(10, TimeUnit.SECONDS); - System.out.println("**** Node 3 Connected ****"); - - final Node clusterCoordinator = cluster.waitForClusterCoordinator(10, TimeUnit.SECONDS); - final Node primaryNode = cluster.waitForPrimaryNode(10, TimeUnit.SECONDS); - System.out.println("\n\n"); - System.out.println("Cluster Coordinator = " + clusterCoordinator); - System.out.println("Primary Node = " + primaryNode); - System.out.println("\n\n"); - } finally { - cluster.stop(); - } + cluster.createNode(); + cluster.createNode(); + cluster.createNode(); + + cluster.waitUntilAllNodesConnected(10, TimeUnit.SECONDS); + + final Node clusterCoordinator = cluster.waitForClusterCoordinator(10, TimeUnit.SECONDS); + final Node primaryNode = cluster.waitForPrimaryNode(10, TimeUnit.SECONDS); + System.out.println("\n\n"); + System.out.println("Cluster Coordinator = " + clusterCoordinator); + System.out.println("Primary Node = " + primaryNode); + System.out.println("\n\n"); } @Test(timeout = 60000) public void testNewCoordinatorElected() throws IOException { - final Cluster cluster = new Cluster(); - cluster.start(); - - try { - final Node firstNode = cluster.createNode(); - final Node secondNode = cluster.createNode(); + final Node firstNode = cluster.createNode(); + final Node secondNode = cluster.createNode(); - firstNode.waitUntilConnected(10, TimeUnit.SECONDS); - System.out.println("**** Node 1 Connected ****"); - secondNode.waitUntilConnected(10, TimeUnit.SECONDS); - System.out.println("**** Node 2 Connected ****"); + cluster.waitUntilAllNodesConnected(10, TimeUnit.SECONDS); - final Node clusterCoordinator = cluster.waitForClusterCoordinator(10, TimeUnit.SECONDS); - clusterCoordinator.stop(); + final Node clusterCoordinator = cluster.waitForClusterCoordinator(10, TimeUnit.SECONDS); + clusterCoordinator.stop(); - final Node otherNode = firstNode == clusterCoordinator ? secondNode : firstNode; - otherNode.waitUntilElectedForRole(ClusterRoles.CLUSTER_COORDINATOR, 10, TimeUnit.SECONDS); - } finally { - cluster.stop(); - } + final Node otherNode = firstNode == clusterCoordinator ? secondNode : firstNode; + otherNode.waitUntilElectedForRole(ClusterRoles.CLUSTER_COORDINATOR, 10, TimeUnit.SECONDS); } @Test(timeout = 60000) public void testReconnectGetsCorrectClusterTopology() throws IOException { - final Cluster cluster = new Cluster(); - cluster.start(); + final Node firstNode = cluster.createNode(); + final Node secondNode = cluster.createNode(); + final Node thirdNode = cluster.createNode(); - try { - final Node firstNode = cluster.createNode(); - final Node secondNode = cluster.createNode(); - final Node thirdNode = cluster.createNode(); - - firstNode.waitUntilConnected(10, TimeUnit.SECONDS); - System.out.println("**** Node 1 Connected ****"); - secondNode.waitUntilConnected(10, TimeUnit.SECONDS); - System.out.println("**** Node 2 Connected ****"); - thirdNode.waitUntilConnected(10, TimeUnit.SECONDS); - System.out.println("**** Node 3 Connected ****"); - - // shutdown node - secondNode.stop(); - - System.out.println("\n\nNode 2 Shut Down\n\n"); - - // wait for node 1 and 3 to recognize that node 2 is gone - Stream.of(firstNode, thirdNode).forEach(node -> { - node.assertNodeDisconnects(secondNode.getIdentifier(), 10, TimeUnit.SECONDS); - }); - - // restart node - secondNode.start(); - System.out.println("\n\nNode 2 Restarted\n\n"); - - secondNode.waitUntilConnected(20, TimeUnit.SECONDS); - System.out.println("\n\nNode 2 Reconnected\n\n"); - - // wait for all 3 nodes to agree that node 2 is connected - Stream.of(firstNode, secondNode, thirdNode).forEach(node -> { - ClusterUtils.waitUntilConditionMet(5, TimeUnit.SECONDS, - () -> firstNode.getClusterCoordinator().getConnectionStatus(secondNode.getIdentifier()).getState() == NodeConnectionState.CONNECTED); - }); - - // Ensure that all 3 nodes see a cluster of 3 connected nodes. - Stream.of(firstNode, secondNode, thirdNode).forEach(node -> { - node.assertNodeIsConnected(firstNode.getIdentifier()); - node.assertNodeIsConnected(secondNode.getIdentifier()); - node.assertNodeIsConnected(thirdNode.getIdentifier()); - }); - - // Ensure that we get both a cluster coordinator and a primary node elected - cluster.waitForClusterCoordinator(10, TimeUnit.SECONDS); - cluster.waitForPrimaryNode(10, TimeUnit.SECONDS); - } finally { - cluster.stop(); - } + cluster.waitUntilAllNodesConnected(10, TimeUnit.SECONDS); + + // shutdown node + secondNode.stop(); + + System.out.println("\n\nNode 2 Shut Down\n\n"); + + // wait for node 1 and 3 to recognize that node 2 is gone + Stream.of(firstNode, thirdNode).forEach(node -> { + node.assertNodeDisconnects(secondNode.getIdentifier(), 10, TimeUnit.SECONDS); + }); + + // restart node + secondNode.start(); + System.out.println("\n\nNode 2 Restarted\n\n"); + + secondNode.waitUntilConnected(20, TimeUnit.SECONDS); + System.out.println("\n\nNode 2 Reconnected\n\n"); + + // wait for all 3 nodes to agree that node 2 is connected + Stream.of(firstNode, secondNode, thirdNode).forEach(node -> { + ClusterUtils.waitUntilConditionMet(5, TimeUnit.SECONDS, + () -> firstNode.getClusterCoordinator().getConnectionStatus(secondNode.getIdentifier()).getState() == NodeConnectionState.CONNECTED); + }); + + // Ensure that all 3 nodes see a cluster of 3 connected nodes. + Stream.of(firstNode, secondNode, thirdNode).forEach(node -> { + node.assertNodeIsConnected(firstNode.getIdentifier()); + node.assertNodeIsConnected(secondNode.getIdentifier()); + node.assertNodeIsConnected(thirdNode.getIdentifier()); + }); + + // Ensure that we get both a cluster coordinator and a primary node elected + cluster.waitForClusterCoordinator(10, TimeUnit.SECONDS); + cluster.waitForPrimaryNode(10, TimeUnit.SECONDS); } @Test(timeout = 60000) public void testRestartAllNodes() throws IOException { - final Cluster cluster = new Cluster(); - cluster.start(); - - try { - final Node firstNode = cluster.createNode(); - final Node secondNode = cluster.createNode(); - final Node thirdNode = cluster.createNode(); - - firstNode.waitUntilConnected(10, TimeUnit.SECONDS); - System.out.println("**** Node 1 Connected ****"); - secondNode.waitUntilConnected(10, TimeUnit.SECONDS); - System.out.println("**** Node 2 Connected ****"); - thirdNode.waitUntilConnected(10, TimeUnit.SECONDS); - System.out.println("**** Node 3 Connected ****"); - - // shutdown node - firstNode.stop(); - secondNode.stop(); - thirdNode.stop(); - - System.out.println("\n\nRestarting all nodes\n\n"); - thirdNode.start(); - firstNode.start(); - secondNode.start(); - - Stream.of(firstNode, secondNode, thirdNode).forEach(node -> { - node.waitUntilConnected(10, TimeUnit.SECONDS); - }); - - // wait for all 3 nodes to agree that node 2 is connected - Stream.of(firstNode, secondNode, thirdNode).forEach(node -> { - ClusterUtils.waitUntilConditionMet(5, TimeUnit.SECONDS, - () -> firstNode.getClusterCoordinator().getConnectionStatus(secondNode.getIdentifier()).getState() == NodeConnectionState.CONNECTED); - }); - - // Ensure that all 3 nodes see a cluster of 3 connected nodes. - Stream.of(firstNode, secondNode, thirdNode).forEach(node -> { - node.assertNodeConnects(firstNode.getIdentifier(), 10, TimeUnit.SECONDS); - node.assertNodeConnects(secondNode.getIdentifier(), 10, TimeUnit.SECONDS); - node.assertNodeConnects(thirdNode.getIdentifier(), 10, TimeUnit.SECONDS); - }); - - // Ensure that we get both a cluster coordinator and a primary node elected - cluster.waitForClusterCoordinator(10, TimeUnit.SECONDS); - cluster.waitForPrimaryNode(10, TimeUnit.SECONDS); - } finally { - cluster.stop(); - } + final Node firstNode = cluster.createNode(); + final Node secondNode = cluster.createNode(); + final Node thirdNode = cluster.createNode(); + + firstNode.waitUntilConnected(10, TimeUnit.SECONDS); + System.out.println("**** Node 1 Connected ****"); + secondNode.waitUntilConnected(10, TimeUnit.SECONDS); + System.out.println("**** Node 2 Connected ****"); + thirdNode.waitUntilConnected(10, TimeUnit.SECONDS); + System.out.println("**** Node 3 Connected ****"); + + // shutdown node + firstNode.stop(); + secondNode.stop(); + thirdNode.stop(); + + System.out.println("\n\nRestarting all nodes\n\n"); + thirdNode.start(); + firstNode.start(); + secondNode.start(); + + cluster.waitUntilAllNodesConnected(10, TimeUnit.SECONDS); + + // wait for all 3 nodes to agree that node 2 is connected + Stream.of(firstNode, secondNode, thirdNode).forEach(node -> { + ClusterUtils.waitUntilConditionMet(5, TimeUnit.SECONDS, + () -> firstNode.getClusterCoordinator().getConnectionStatus(secondNode.getIdentifier()).getState() == NodeConnectionState.CONNECTED); + }); + + // Ensure that all 3 nodes see a cluster of 3 connected nodes. + Stream.of(firstNode, secondNode, thirdNode).forEach(node -> { + node.assertNodeConnects(firstNode.getIdentifier(), 10, TimeUnit.SECONDS); + node.assertNodeConnects(secondNode.getIdentifier(), 10, TimeUnit.SECONDS); + node.assertNodeConnects(thirdNode.getIdentifier(), 10, TimeUnit.SECONDS); + }); + + // Ensure that we get both a cluster coordinator and a primary node elected + cluster.waitForClusterCoordinator(10, TimeUnit.SECONDS); + cluster.waitForPrimaryNode(10, TimeUnit.SECONDS); } @Test(timeout = 30000) public void testHeartbeatsMonitored() throws IOException { - final Cluster cluster = new Cluster(); - cluster.start(); + final Node firstNode = cluster.createNode(); + final Node secondNode = cluster.createNode(); - try { - final Node firstNode = cluster.createNode(); - final Node secondNode = cluster.createNode(); + cluster.waitUntilAllNodesConnected(10, TimeUnit.SECONDS); - firstNode.waitUntilConnected(10, TimeUnit.SECONDS); - secondNode.waitUntilConnected(10, TimeUnit.SECONDS); + final Node nodeToSuspend = firstNode; + final Node otherNode = secondNode; - final Node nodeToSuspend = firstNode; - final Node otherNode = secondNode; + nodeToSuspend.suspendHeartbeating(); - nodeToSuspend.suspendHeartbeating(); + // Heartbeat interval in nifi.properties is set to 1 sec. This means that the node should be kicked out + // due to lack of heartbeat after 8 times this amount of time, or 8 seconds. + otherNode.assertNodeDisconnects(nodeToSuspend.getIdentifier(), 12, TimeUnit.SECONDS); - // Heartbeat interval in nifi.properties is set to 1 sec. This means that the node should be kicked out - // due to lack of heartbeat after 8 times this amount of time, or 8 seconds. - otherNode.assertNodeDisconnects(nodeToSuspend.getIdentifier(), 12, TimeUnit.SECONDS); + nodeToSuspend.resumeHeartbeating(); + otherNode.assertNodeConnects(nodeToSuspend.getIdentifier(), 10, TimeUnit.SECONDS); + } - nodeToSuspend.resumeHeartbeating(); - otherNode.assertNodeConnects(nodeToSuspend.getIdentifier(), 10, TimeUnit.SECONDS); - } finally { - cluster.stop(); + @Test + public void testNodeInheritsClusterTopologyOnHeartbeat() throws InterruptedException { + final Node node1 = cluster.createNode(); + final Node node2 = cluster.createNode(); + final Node node3 = cluster.createNode(); + + cluster.waitUntilAllNodesConnected(10, TimeUnit.SECONDS); + final Node coordinator = cluster.waitForClusterCoordinator(10, TimeUnit.SECONDS); + + final NodeIdentifier node4NotReallyInCluster = new NodeIdentifier(UUID.randomUUID().toString(), "localhost", 9283, "localhost", 9284, "localhost", 9285, null, false, null); + + final Map replacementStatuses = new HashMap<>(); + replacementStatuses.put(node1.getIdentifier(), new NodeConnectionStatus(node1.getIdentifier(), DisconnectionCode.USER_DISCONNECTED)); + replacementStatuses.put(node4NotReallyInCluster, new NodeConnectionStatus(node4NotReallyInCluster, NodeConnectionState.CONNECTING)); + + // reset coordinator status so that other nodes with get its now-fake view of the cluster + coordinator.getClusterCoordinator().resetNodeStatuses(replacementStatuses); + final List expectedStatuses = coordinator.getClusterCoordinator().getConnectionStatuses(); + + // give nodes a bit to heartbeat in. We need to wait long enough that each node heartbeats. + // But we need to not wait more than 8 seconds because that's when nodes start getting kicked out. + Thread.sleep(6000L); + + for (final Node node : new Node[] {node1, node2, node3}) { + assertEquals(expectedStatuses, node.getClusterCoordinator().getConnectionStatuses()); } } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java index 4fb5aef5e858..b97440625ba7 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java @@ -65,7 +65,6 @@ import org.apache.nifi.authorization.resource.DataAuthorizable; import org.apache.nifi.authorization.resource.ResourceFactory; import org.apache.nifi.authorization.user.NiFiUser; -import org.apache.nifi.cluster.HeartbeatPayload; import org.apache.nifi.cluster.coordination.ClusterCoordinator; import org.apache.nifi.cluster.coordination.heartbeat.HeartbeatMonitor; import org.apache.nifi.cluster.coordination.node.ClusterRoles; @@ -74,6 +73,7 @@ import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; import org.apache.nifi.cluster.protocol.DataFlow; import org.apache.nifi.cluster.protocol.Heartbeat; +import org.apache.nifi.cluster.protocol.HeartbeatPayload; import org.apache.nifi.cluster.protocol.NodeIdentifier; import org.apache.nifi.cluster.protocol.NodeProtocolSender; import org.apache.nifi.cluster.protocol.UnknownServiceAddressException; @@ -586,7 +586,7 @@ public void run() { this.leaderElectionManager = leaderElectionManager; if (configuredForClustering) { - heartbeater = new ClusterProtocolHeartbeater(protocolSender, leaderElectionManager); + heartbeater = new ClusterProtocolHeartbeater(protocolSender, clusterCoordinator, leaderElectionManager); // Check if there is already a cluster coordinator elected. If not, go ahead // and register for coordinator role. If there is already one elected, do not register until @@ -3854,7 +3854,7 @@ private class HeartbeatSendTask implements Runnable { @Override public void run() { - try { + try (final NarCloseable narCloseable = NarCloseable.withFrameworkNar()) { if (heartbeatsSuspended.get()) { return; } @@ -3916,6 +3916,7 @@ HeartbeatMessage createHeartbeatMessage() { final QueueSize queueSize = getTotalFlowFileCount(bean.getRootGroup()); hbPayload.setTotalFlowFileCount(queueSize.getObjectCount()); hbPayload.setTotalFlowFileBytes(queueSize.getByteCount()); + hbPayload.setClusterStatus(clusterCoordinator.getConnectionStatuses()); // create heartbeat message final NodeIdentifier nodeId = getNodeId(); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/cluster/ClusterProtocolHeartbeater.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/cluster/ClusterProtocolHeartbeater.java index eb3cebe73b53..d675d0c132f5 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/cluster/ClusterProtocolHeartbeater.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/cluster/ClusterProtocolHeartbeater.java @@ -18,12 +18,22 @@ package org.apache.nifi.controller.cluster; import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.nifi.cluster.coordination.ClusterCoordinator; import org.apache.nifi.cluster.coordination.node.ClusterRoles; +import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; +import org.apache.nifi.cluster.protocol.HeartbeatPayload; +import org.apache.nifi.cluster.protocol.NodeIdentifier; import org.apache.nifi.cluster.protocol.NodeProtocolSender; import org.apache.nifi.cluster.protocol.ProtocolException; import org.apache.nifi.cluster.protocol.message.HeartbeatMessage; +import org.apache.nifi.cluster.protocol.message.HeartbeatResponseMessage; import org.apache.nifi.controller.leader.election.LeaderElectionManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Uses Leader Election Manager in order to determine which node is the elected Cluster Coordinator and to indicate @@ -31,11 +41,15 @@ * sent directly to the Cluster Coordinator. */ public class ClusterProtocolHeartbeater implements Heartbeater { + private static final Logger logger = LoggerFactory.getLogger(ClusterProtocolHeartbeater.class); + private final NodeProtocolSender protocolSender; private final LeaderElectionManager electionManager; + private final ClusterCoordinator clusterCoordinator; - public ClusterProtocolHeartbeater(final NodeProtocolSender protocolSender, final LeaderElectionManager electionManager) { + public ClusterProtocolHeartbeater(final NodeProtocolSender protocolSender, final ClusterCoordinator clusterCoordinator, final LeaderElectionManager electionManager) { this.protocolSender = protocolSender; + this.clusterCoordinator = clusterCoordinator; this.electionManager = electionManager; } @@ -52,7 +66,28 @@ public String getHeartbeatAddress() throws IOException { @Override public synchronized void send(final HeartbeatMessage heartbeatMessage) throws IOException { final String heartbeatAddress = getHeartbeatAddress(); - protocolSender.heartbeat(heartbeatMessage, heartbeatAddress); + final HeartbeatResponseMessage responseMessage = protocolSender.heartbeat(heartbeatMessage, heartbeatAddress); + + final byte[] payloadBytes = heartbeatMessage.getHeartbeat().getPayload(); + final HeartbeatPayload payload = HeartbeatPayload.unmarshal(payloadBytes); + final List nodeStatusList = payload.getClusterStatus(); + final Map updateIdMap = nodeStatusList.stream().collect( + Collectors.toMap(status -> status.getNodeIdentifier(), status -> status.getUpdateIdentifier())); + + final List updatedStatuses = responseMessage.getUpdatedNodeStatuses(); + if (updatedStatuses != null) { + for (final NodeConnectionStatus updatedStatus : updatedStatuses) { + final NodeIdentifier nodeId = updatedStatus.getNodeIdentifier(); + final Long updateId = updateIdMap.get(nodeId); + + final boolean updated = clusterCoordinator.resetNodeStatus(updatedStatus, updateId == null ? -1L : updateId); + if (updated) { + logger.info("After receiving heartbeat response, updated status of {} to {}", updatedStatus.getNodeIdentifier(), updatedStatus); + } else { + logger.debug("After receiving heartbeat response, did not update status of {} to {} because the update is out-of-date", updatedStatus.getNodeIdentifier(), updatedStatus); + } + } + } } @Override diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/cluster/HeartbeatPayloadTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/cluster/HeartbeatPayloadTest.java index af73eef09494..429a791f5e55 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/cluster/HeartbeatPayloadTest.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/cluster/HeartbeatPayloadTest.java @@ -21,6 +21,7 @@ import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; +import org.apache.nifi.cluster.protocol.HeartbeatPayload; import org.apache.nifi.util.NiFiProperties; import org.junit.Before; import org.junit.BeforeClass; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarCloseable.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarCloseable.java index b25c90b65601..cbb96b105cde 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarCloseable.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarCloseable.java @@ -18,10 +18,14 @@ import java.io.Closeable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * */ public class NarCloseable implements Closeable { + private static final Logger logger = LoggerFactory.getLogger(NarCloseable.class); public static NarCloseable withNarLoader() { final ClassLoader current = Thread.currentThread().getContextClassLoader(); @@ -29,6 +33,31 @@ public static NarCloseable withNarLoader() { return new NarCloseable(current); } + /** + * Creates a Closeable object that can be used to to switch to current class loader to the framework class loader + * and will automatically set the ClassLoader back to the previous class loader when closed + * + * @return a NarCloseable + */ + public static NarCloseable withFrameworkNar() { + final ClassLoader frameworkClassLoader; + try { + frameworkClassLoader = NarClassLoaders.getInstance().getFrameworkClassLoader(); + } catch (final Exception e) { + // This should never happen in a running instance, but it will occur in unit tests + logger.error("Unable to access Framework ClassLoader due to " + e + ". Will continue without change ClassLoaders."); + if (logger.isDebugEnabled()) { + logger.error("", e); + } + + return new NarCloseable(null); + } + + final ClassLoader current = Thread.currentThread().getContextClassLoader(); + Thread.currentThread().setContextClassLoader(frameworkClassLoader); + return new NarCloseable(current); + } + private final ClassLoader toSet; private NarCloseable(final ClassLoader toSet) { From 8ca8d0151a01aa1f8c11c459b7f3b708b67af75c Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Tue, 16 Aug 2016 10:57:41 -0400 Subject: [PATCH 4/4] NIFI-2566: Fixed issue that prevented standalone instance from starting by creating a standalone-instance version of the Leader Election Manager. Also added Controller Service enabled/disabled state to fingerprint rather than attempting to update the state when joining the cluster, as the implementation was incorrect and the correct implementation will be a rather significant effort that doesn't have to happen for 1.0.0 release --- .../nifi-cluster-manager-context.xml | 6 +- .../controller/StandardFlowSynchronizer.java | 83 +++++++------------ .../StandaloneLeaderElectionManager.java | 65 +++++++++++++++ .../nifi/fingerprint/FingerprintFactory.java | 1 + .../LeaderElectionManagerFactoryBean.java | 57 +++++++++++++ 5 files changed, 154 insertions(+), 58 deletions(-) create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/leader/election/StandaloneLeaderElectionManager.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/LeaderElectionManagerFactoryBean.java diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/resources/nifi-cluster-manager-context.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/resources/nifi-cluster-manager-context.xml index 38b3a5be18f6..84c9deb8ea69 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/resources/nifi-cluster-manager-context.xml +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/resources/nifi-cluster-manager-context.xml @@ -36,9 +36,9 @@ - - - + + + diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java index d1822efebf12..cb663e15b130 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java @@ -16,6 +16,34 @@ */ package org.apache.nifi.controller; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.net.URL; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardOpenOption; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.zip.GZIPInputStream; + +import javax.xml.XMLConstants; +import javax.xml.parsers.DocumentBuilder; +import javax.xml.parsers.DocumentBuilderFactory; +import javax.xml.parsers.ParserConfigurationException; +import javax.xml.validation.Schema; +import javax.xml.validation.SchemaFactory; + import org.apache.commons.collections4.CollectionUtils; import org.apache.nifi.authorization.AbstractPolicyBasedAuthorizer; import org.apache.nifi.authorization.Authorizer; @@ -41,7 +69,6 @@ import org.apache.nifi.controller.serialization.StandardFlowSerializer; import org.apache.nifi.controller.service.ControllerServiceLoader; import org.apache.nifi.controller.service.ControllerServiceNode; -import org.apache.nifi.controller.service.ControllerServiceState; import org.apache.nifi.encrypt.StringEncryptor; import org.apache.nifi.events.BulletinFactory; import org.apache.nifi.fingerprint.FingerprintException; @@ -66,7 +93,6 @@ import org.apache.nifi.util.file.FileUtils; import org.apache.nifi.web.api.dto.ConnectableDTO; import org.apache.nifi.web.api.dto.ConnectionDTO; -import org.apache.nifi.web.api.dto.ControllerServiceDTO; import org.apache.nifi.web.api.dto.FlowSnippetDTO; import org.apache.nifi.web.api.dto.FunnelDTO; import org.apache.nifi.web.api.dto.LabelDTO; @@ -86,33 +112,6 @@ import org.w3c.dom.NodeList; import org.xml.sax.SAXException; -import javax.xml.XMLConstants; -import javax.xml.parsers.DocumentBuilder; -import javax.xml.parsers.DocumentBuilderFactory; -import javax.xml.parsers.ParserConfigurationException; -import javax.xml.validation.Schema; -import javax.xml.validation.SchemaFactory; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.InputStream; -import java.net.URL; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.StandardOpenOption; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; -import java.util.zip.GZIPInputStream; - /** */ public class StandardFlowSynchronizer implements FlowSynchronizer { @@ -360,10 +359,6 @@ public void sync(final FlowController controller, final DataFlow proposedFlow, f // enable all the original controller services ControllerServiceLoader.enableControllerServices(controllerServices, controller, encryptor, autoResumeState); - } else { - for (final Element serviceElement : serviceElements) { - updateControllerService(controller, serviceElement, encryptor); - } } } @@ -505,22 +500,6 @@ private byte[] readFlowFromDisk() throws IOException { return baos.toByteArray(); } - private void updateControllerService(final FlowController controller, final Element controllerServiceElement, final StringEncryptor encryptor) { - final ControllerServiceDTO dto = FlowFromDOMFactory.getControllerService(controllerServiceElement, encryptor); - - final ControllerServiceState dtoState = ControllerServiceState.valueOf(dto.getState()); - final boolean dtoEnabled = (dtoState == ControllerServiceState.ENABLED || dtoState == ControllerServiceState.ENABLING); - - final ControllerServiceNode serviceNode = controller.getControllerServiceNode(dto.getId()); - final ControllerServiceState serviceState = serviceNode.getState(); - final boolean serviceEnabled = (serviceState == ControllerServiceState.ENABLED || serviceState == ControllerServiceState.ENABLING); - - if (dtoEnabled && !serviceEnabled) { - controller.enableControllerService(controller.getControllerServiceNode(dto.getId())); - } else if (!dtoEnabled && serviceEnabled) { - controller.disableControllerService(controller.getControllerServiceNode(dto.getId())); - } - } private ReportingTaskNode getOrCreateReportingTask(final FlowController controller, final ReportingTaskDTO dto, final boolean controllerInitialized, final boolean existingFlowEmpty) throws ReportingTaskInstantiationException { @@ -665,12 +644,6 @@ private ProcessGroup updateProcessGroup(final FlowController controller, final P // get the real process group and ID final ProcessGroup processGroup = controller.getGroup(processGroupDto.getId()); - // Update Controller Services - final List serviceNodeList = getChildrenByTagName(processGroupElement, "controllerService"); - for (final Element serviceNodeElement : serviceNodeList) { - updateControllerService(controller, serviceNodeElement, encryptor); - } - // processors & ports cannot be updated - they must be the same. Except for the scheduled state. final List processorNodeList = getChildrenByTagName(processGroupElement, "processor"); for (final Element processorElement : processorNodeList) { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/leader/election/StandaloneLeaderElectionManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/leader/election/StandaloneLeaderElectionManager.java new file mode 100644 index 000000000000..a2ed86e94d29 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/leader/election/StandaloneLeaderElectionManager.java @@ -0,0 +1,65 @@ +/* + * 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.nifi.controller.leader.election; + +/** + *

+ * A LeaderElectionManager to use when running a standalone (un-clustered) NiFi instance + *

+ */ +public class StandaloneLeaderElectionManager implements LeaderElectionManager { + + @Override + public void start() { + } + + @Override + public void register(final String roleName) { + } + + @Override + public void register(final String roleName, final LeaderElectionStateChangeListener listener) { + } + + @Override + public void register(final String roleName, final LeaderElectionStateChangeListener listener, final String participantId) { + } + + @Override + public String getLeader(final String roleName) { + return null; + } + + @Override + public void unregister(final String roleName) { + } + + @Override + public boolean isLeader(final String roleName) { + return false; + } + + @Override + public boolean isStopped() { + return false; + } + + @Override + public void stop() { + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java index 88fbcdd96aa9..d50d31b1cc39 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java @@ -908,6 +908,7 @@ private void addControllerServiceFingerprint(final StringBuilder builder, final builder.append(dto.getName()); builder.append(dto.getComments()); builder.append(dto.getAnnotationData()); + builder.append(dto.getState()); final Map properties = dto.getProperties(); if (properties == null) { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/LeaderElectionManagerFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/LeaderElectionManagerFactoryBean.java new file mode 100644 index 000000000000..f17cf1b8f060 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/LeaderElectionManagerFactoryBean.java @@ -0,0 +1,57 @@ +/* + * 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.nifi.spring; + +import org.apache.nifi.controller.leader.election.CuratorLeaderElectionManager; +import org.apache.nifi.controller.leader.election.LeaderElectionManager; +import org.apache.nifi.controller.leader.election.StandaloneLeaderElectionManager; +import org.apache.nifi.util.NiFiProperties; +import org.springframework.beans.factory.FactoryBean; + +public class LeaderElectionManagerFactoryBean implements FactoryBean { + private int numThreads; + private NiFiProperties properties; + + @Override + public LeaderElectionManager getObject() throws Exception { + final boolean isNode = properties.isNode(); + if (isNode) { + return new CuratorLeaderElectionManager(numThreads, properties); + } else { + return new StandaloneLeaderElectionManager(); + } + } + + @Override + public Class getObjectType() { + return LeaderElectionManager.class; + } + + @Override + public boolean isSingleton() { + return true; + } + + public void setNumThreads(final int numThreads) { + this.numThreads = numThreads; + } + + public void setProperties(final NiFiProperties properties) { + this.properties = properties; + } +}