emptyMap());
}
@Override
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ComponentDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ComponentDTO.java
index ca5e06fa1d..0ef7fef4e4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ComponentDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ComponentDTO.java
@@ -91,4 +91,22 @@ public class ComponentDTO {
public void setPosition(final PositionDTO position) {
this.position = position;
}
+
+ @Override
+ public int hashCode() {
+ return id == null ? 187 : 187 * id.hashCode();
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj == null) {
+ return false;
+ }
+
+ if (obj.getClass() != getClass()) {
+ return false;
+ }
+
+ return id.equals(((ComponentDTO) obj).getId());
+ }
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ComponentEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ComponentEntity.java
index ab9889900c..2f7798ee73 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ComponentEntity.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ComponentEntity.java
@@ -112,4 +112,26 @@ public class ComponentEntity extends Entity {
public void setAccessPolicy(AccessPolicyDTO accessPolicy) {
this.accessPolicy = accessPolicy;
}
+
+ @Override
+ public int hashCode() {
+ return id.hashCode();
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj == null) {
+ return false;
+ }
+
+ if (obj == this) {
+ return true;
+ }
+
+ if (obj.getClass() != getClass()) {
+ return false;
+ }
+
+ return id.equals(((ComponentEntity) obj).getId());
+ }
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/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
similarity index 64%
rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/ClusterCoordinator.java
rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/ClusterCoordinator.java
index fa49a62ba9..7cd9f8068f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/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
@@ -24,8 +24,10 @@ 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.coordination.node.NodeConnectionStatus;
+import org.apache.nifi.cluster.event.NodeEvent;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.reporting.Severity;
+import org.apache.nifi.services.FlowService;
/**
*
@@ -36,11 +38,18 @@ public interface ClusterCoordinator {
/**
* Sends a request to the node to connect to the cluster. This will immediately
- * set the NodeConnectionStatus to DISCONNECTED.
+ * set the NodeConnectionStatus to CONNECTING.
*
* @param nodeId the identifier of the node
+ * @param userDn the DN of the user that requested that the node connect, or null
if the action is not user-driven
*/
- void requestNodeConnect(NodeIdentifier nodeId);
+ void requestNodeConnect(NodeIdentifier nodeId, String userDn);
+
+ /**
+ * Notifies the Cluster Coordinator that the NiFi instance is being shutdown so that
+ * the coordinator is able to perform cleanup of resources
+ */
+ void shutdown();
/**
* Indicates that the node has sent a valid heartbeat and should now
@@ -71,6 +80,14 @@ public interface ClusterCoordinator {
*/
void disconnectionRequestedByNode(NodeIdentifier nodeId, DisconnectionCode disconnectionCode, String explanation);
+ /**
+ * Removes the given disconnected node from the cluster
+ *
+ * @param nodeId the node to remove
+ * @param userDn the DN of the user requesting that the node be removed
+ */
+ void removeNode(NodeIdentifier nodeId, String userDn);
+
/**
* Returns the current status of the node with the given identifier
*
@@ -84,10 +101,10 @@ public interface ClusterCoordinator {
/**
* Returns the identifiers of all nodes that have the given connection state
*
- * @param state the state
+ * @param states the states of interest
* @return the identifiers of all nodes that have the given connection state
*/
- Set getNodeIdentifiers(NodeConnectionState state);
+ Set getNodeIdentifiers(NodeConnectionState... states);
/**
* Returns a Map of NodeConnectionStatus to all Node Identifiers that have that status.
@@ -118,11 +135,12 @@ public interface ClusterCoordinator {
void reportEvent(NodeIdentifier nodeId, Severity severity, String event);
/**
- * Updates the node that is considered the Primary Node
+ * Updates the roles held by the given node
*
- * @param nodeId the id of the Primary Node
+ * @param nodeId the id of the node to update
+ * @param roles the new roles that the node possesses
*/
- void setPrimaryNode(NodeIdentifier nodeId);
+ void updateNodeRoles(NodeIdentifier nodeId, Set roles);
/**
* Returns the NodeIdentifier that exists that has the given UUID, or null
if no NodeIdentifier
@@ -133,4 +151,53 @@ public interface ClusterCoordinator {
* exists for the given UUID
*/
NodeIdentifier getNodeIdentifier(String uuid);
+
+ /**
+ * Returns all of the events that have occurred for the given node
+ *
+ * @param nodeId the identifier of the node
+ * @return all of the events that have occurred for the given node
+ */
+ List getNodeEvents(NodeIdentifier nodeId);
+
+ /**
+ * @return the identifier of the node that is elected primary, or null
if either there is no
+ * primary or the primary is not known by this node.
+ */
+ NodeIdentifier getPrimaryNode();
+
+ /**
+ * Updates the Flow Service to use for obtaining the current flow
+ *
+ * @param flowService the flow service to use for obtaining the current flow
+ */
+ void setFlowService(FlowService flowService);
+
+ /**
+ * Clears the current state of all nodes and replaces them with the values provided in the given map
+ *
+ * @param statusMap the new states of all nodes in the cluster
+ */
+ void resetNodeStatuses(Map statusMap);
+
+ /**
+ * Notifies the Cluster Coordinator of the Node Identifier that the coordinator is currently running on
+ *
+ * @param nodeId the ID of the current node
+ */
+ void setLocalNodeIdentifier(NodeIdentifier nodeId);
+
+ /**
+ * Notifies the Cluster Coordinator whether or not the node is connected to the cluster
+ *
+ * @param connected true
if the node is connected to a cluster, false
otherwise.
+ */
+ void setConnected(boolean connected);
+
+ /**
+ * Indicates whether or not the node is currently connected to the cluster
+ *
+ * @return true
if connected, false
otherwise
+ */
+ boolean isConnected();
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/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
similarity index 100%
rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/HeartbeatMonitor.java
rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/HeartbeatMonitor.java
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/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
similarity index 93%
rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/NodeHeartbeat.java
rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/NodeHeartbeat.java
index bd66022d23..2ddda79012 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/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,6 +17,8 @@
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;
@@ -38,9 +40,9 @@ public interface NodeHeartbeat {
NodeConnectionStatus getConnectionStatus();
/**
- * @return true
if the node is the Primary Node in the cluster, false
otherwise
+ * @return the set of Roles that the node currently possesses.
*/
- boolean isPrimary();
+ 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/src/main/java/org/apache/nifi/cluster/flow/PersistedFlowState.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
similarity index 64%
rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/PersistedFlowState.java
rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/ClusterRoles.java
index 166632ca92..611faa40f8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/PersistedFlowState.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
@@ -14,23 +14,12 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.nifi.cluster.flow;
-/**
- * Represents the various state of a flow managed by the cluster.
- *
- * The semantics of the values are:
- *
- * - CURRENT - the flow is current
- * - STALE - the flow is not current, but is eligible to be updated.
- * - UNKNOWN - the flow is not current and is not eligible to be updated.
- *
- *
- *
- */
-public enum PersistedFlowState {
+package org.apache.nifi.cluster.coordination.node;
- CURRENT,
- STALE,
- UNKNOWN
+public class ClusterRoles {
+
+ public static final String PRIMARY_NODE = "Primary Node";
+
+ public static final String CLUSTER_COORDINATOR = "Cluster Coordinator";
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/DisconnectionCode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/DisconnectionCode.java
index bd20c3f216..ae18699c38 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/DisconnectionCode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/DisconnectionCode.java
@@ -57,6 +57,16 @@ public enum DisconnectionCode {
*/
MISMATCHED_FLOWS("Node's Flow did not Match Cluster Flow"),
+ /**
+ * Cannot communicate with the node
+ */
+ UNABLE_TO_COMMUNICATE("Unable to Communicate with Node"),
+
+ /**
+ * Node did not service a request that was replicated to it
+ */
+ FAILED_TO_SERVICE_REQUEST("Failed to Service Request"),
+
/**
* Node is being shut down
*/
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/NodeConnectionState.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/NodeConnectionState.java
index 9627b2d7aa..8d5824f171 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/NodeConnectionState.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/NodeConnectionState.java
@@ -46,5 +46,11 @@ public enum NodeConnectionState {
* A node that is not connected to the cluster.
* A DISCONNECTED node can transition to CONNECTING.
*/
- DISCONNECTED
+ DISCONNECTED,
+
+ /**
+ * A NodeConnectionState of REMOVED indicates that the node was removed from the cluster
+ * and is used in order to notify other nodes in the cluster.
+ */
+ REMOVED;
}
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 cd2a6a7d5d..4570a241e8 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,8 +17,15 @@
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;
+import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.cluster.protocol.jaxb.message.NodeConnectionStatusAdapter;
/**
@@ -26,33 +33,55 @@ import org.apache.nifi.cluster.protocol.jaxb.message.NodeConnectionStatusAdapter
*/
@XmlJavaTypeAdapter(NodeConnectionStatusAdapter.class)
public class NodeConnectionStatus {
+ private static final AtomicLong idGenerator = new AtomicLong(0L);
+
+ private final long updateId;
+ private final NodeIdentifier nodeId;
private final NodeConnectionState state;
private final DisconnectionCode disconnectCode;
private final String disconnectReason;
private final Long connectionRequestTime;
+ private final Set roles;
- public NodeConnectionStatus(final NodeConnectionState state) {
- this(state, null, null, null);
+ public NodeConnectionStatus(final NodeIdentifier nodeId, final NodeConnectionState state) {
+ this(nodeId, state, null, null, null, null);
}
- public NodeConnectionStatus(final NodeConnectionState state, final long connectionRequestTime) {
- this(state, null, null, connectionRequestTime);
+ public NodeConnectionStatus(final NodeIdentifier nodeId, final NodeConnectionState state, final Set roles) {
+ this(nodeId, state, null, null, null, roles);
}
- public NodeConnectionStatus(final DisconnectionCode disconnectionCode) {
- this(NodeConnectionState.DISCONNECTED, disconnectionCode, disconnectionCode.name(), null);
+ public NodeConnectionStatus(final NodeIdentifier nodeId, final NodeConnectionState state, final long connectionRequestTime) {
+ this(nodeId, state, null, null, connectionRequestTime, null);
}
- public NodeConnectionStatus(final DisconnectionCode disconnectionCode, final String disconnectionExplanation) {
- this(NodeConnectionState.DISCONNECTED, disconnectionCode, disconnectionExplanation, null);
+ public NodeConnectionStatus(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode) {
+ this(nodeId, NodeConnectionState.DISCONNECTED, disconnectionCode, disconnectionCode.name(), null, null);
}
- public NodeConnectionStatus(final NodeConnectionState state, final DisconnectionCode disconnectionCode) {
- this(state, disconnectionCode, disconnectionCode.name(), null);
+ public NodeConnectionStatus(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode, final String disconnectionExplanation) {
+ this(nodeId, NodeConnectionState.DISCONNECTED, disconnectionCode, disconnectionExplanation, null, null);
}
- public NodeConnectionStatus(final NodeConnectionState state, final DisconnectionCode disconnectCode, final String disconnectReason, final Long connectionRequestTime) {
+ 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 NodeConnectionStatus status, final Set roles) {
+ this(status.getNodeIdentifier(), status.getState(), status.getDisconnectCode(), status.getDisconnectReason(), status.getConnectionRequestTime(), roles);
+ }
+
+ 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);
+ }
+
+ public NodeConnectionStatus(final long updateId, final NodeIdentifier nodeId, final NodeConnectionState state, final DisconnectionCode disconnectCode,
+ final String disconnectReason, final Long connectionRequestTime, final Set roles) {
+ 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();
@@ -61,7 +90,19 @@ public class NodeConnectionStatus {
this.disconnectReason = disconnectReason;
}
- this.connectionRequestTime = connectionRequestTime;
+ this.connectionRequestTime = (connectionRequestTime == null && state == NodeConnectionState.CONNECTING) ? Long.valueOf(System.currentTimeMillis()) : connectionRequestTime;
+ }
+
+ public long getUpdateIdentifier() {
+ return updateId;
+ }
+
+ public Set getRoles() {
+ return roles;
+ }
+
+ public NodeIdentifier getNodeIdentifier() {
+ return nodeId;
}
public NodeConnectionState getState() {
@@ -88,7 +129,47 @@ public class NodeConnectionStatus {
if (state == NodeConnectionState.DISCONNECTED || state == NodeConnectionState.DISCONNECTING) {
sb.append(", Disconnect Code=").append(getDisconnectCode()).append(", Disconnect Reason=").append(getDisconnectReason());
}
+ sb.append(", updateId=").append(getUpdateIdentifier());
sb.append("]");
return sb.toString();
}
+
+ /**
+ * Updates the ID Generator so that it is at least equal to the given minimum value
+ *
+ * @param minimumValue the minimum value that the ID Generator should be set to
+ */
+ static void updateIdGenerator(long minimumValue) {
+ idGenerator.updateAndGet(curValue -> Math.max(minimumValue, curValue));
+ }
+
+ @Override
+ 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;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj) {
+ return true;
+ }
+
+ if (obj == null) {
+ return false;
+ }
+
+ if (!(obj instanceof NodeConnectionStatus)) {
+ return false;
+ }
+
+ 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/src/main/java/org/apache/nifi/cluster/flow/DaoException.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/event/NodeEvent.java
similarity index 66%
rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/DaoException.java
rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/event/NodeEvent.java
index aed86c91ec..15accb6f8b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/DaoException.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/event/NodeEvent.java
@@ -14,26 +14,17 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.nifi.cluster.flow;
-/**
- * A base exception for data access exceptions.
- *
- */
-public class DaoException extends RuntimeException {
+package org.apache.nifi.cluster.event;
- public DaoException() {
- }
+import org.apache.nifi.reporting.Severity;
- public DaoException(String msg) {
- super(msg);
- }
+public interface NodeEvent {
+ Severity getSeverity();
- public DaoException(Throwable cause) {
- super(cause);
- }
+ String getMessage();
- public DaoException(String msg, Throwable cause) {
- super(msg, cause);
- }
+ String getSource();
+
+ long getTimestamp();
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImpl.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/AbstractNodeProtocolSender.java
similarity index 71%
rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImpl.java
rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/AbstractNodeProtocolSender.java
index 23b120901a..7d5f4427c3 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImpl.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/AbstractNodeProtocolSender.java
@@ -14,18 +14,9 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.nifi.cluster.protocol.impl;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.security.cert.CertificateException;
-import org.apache.nifi.cluster.protocol.NodeProtocolSender;
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller;
-import org.apache.nifi.cluster.protocol.ProtocolMessageUnmarshaller;
-import org.apache.nifi.cluster.protocol.UnknownServiceAddressException;
+package org.apache.nifi.cluster.protocol;
+
import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage;
import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage;
import org.apache.nifi.cluster.protocol.message.HeartbeatMessage;
@@ -33,26 +24,18 @@ import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType;
import org.apache.nifi.io.socket.SocketConfiguration;
import org.apache.nifi.io.socket.SocketUtils;
-import org.apache.nifi.io.socket.multicast.DiscoverableService;
import org.apache.nifi.security.util.CertificateUtils;
-public class NodeProtocolSenderImpl implements NodeProtocolSender {
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.security.cert.CertificateException;
+public abstract class AbstractNodeProtocolSender implements NodeProtocolSender {
private final SocketConfiguration socketConfiguration;
- private final ClusterServiceLocator clusterManagerProtocolServiceLocator;
private final ProtocolContext protocolContext;
- public NodeProtocolSenderImpl(final ClusterServiceLocator clusterManagerProtocolServiceLocator,
- final SocketConfiguration socketConfiguration, final ProtocolContext protocolContext) {
- if (clusterManagerProtocolServiceLocator == null) {
- throw new IllegalArgumentException("Protocol Service Locator may not be null.");
- } else if (socketConfiguration == null) {
- throw new IllegalArgumentException("Socket configuration may not be null.");
- } else if (protocolContext == null) {
- throw new IllegalArgumentException("Protocol Context may not be null.");
- }
-
- this.clusterManagerProtocolServiceLocator = clusterManagerProtocolServiceLocator;
+ public AbstractNodeProtocolSender(final SocketConfiguration socketConfiguration, final ProtocolContext protocolContext) {
this.socketConfiguration = socketConfiguration;
this.protocolContext = protocolContext;
}
@@ -63,7 +46,7 @@ public class NodeProtocolSenderImpl implements NodeProtocolSender {
try {
socket = createSocket();
- String ncmDn = getNCMDN(socket);
+ String coordinatorDN = getCoordinatorDN(socket);
try {
// marshal message to output stream
@@ -79,12 +62,13 @@ public class NodeProtocolSenderImpl implements NodeProtocolSender {
final ProtocolMessageUnmarshaller unmarshaller = protocolContext.createUnmarshaller();
response = unmarshaller.unmarshal(socket.getInputStream());
} catch (final IOException ioe) {
- throw new ProtocolException("Failed unmarshalling '" + MessageType.CONNECTION_RESPONSE + "' protocol message due to: " + ioe, ioe);
+ throw new ProtocolException("Failed unmarshalling '" + MessageType.CONNECTION_RESPONSE + "' protocol message from "
+ + socket.getRemoteSocketAddress() + " due to: " + ioe, ioe);
}
if (MessageType.CONNECTION_RESPONSE == response.getType()) {
final ConnectionResponseMessage connectionResponse = (ConnectionResponseMessage) response;
- connectionResponse.setClusterManagerDN(ncmDn);
+ connectionResponse.setCoordinatorDN(coordinatorDN);
return connectionResponse;
} else {
throw new ProtocolException("Expected message type '" + MessageType.CONNECTION_RESPONSE + "' but found '" + response.getType() + "'");
@@ -109,7 +93,7 @@ public class NodeProtocolSenderImpl implements NodeProtocolSender {
sendProtocolMessage(msg, hostname, port);
}
- private String getNCMDN(Socket socket) {
+ private String getCoordinatorDN(Socket socket) {
try {
return CertificateUtils.extractClientDNFromSSLSocket(socket);
} catch (CertificateException e) {
@@ -118,17 +102,17 @@ public class NodeProtocolSenderImpl implements NodeProtocolSender {
}
private Socket createSocket() {
- // determine the cluster manager's address
- final DiscoverableService service = clusterManagerProtocolServiceLocator.getService();
- if (service == null) {
- throw new UnknownServiceAddressException("Cluster Manager's service is not known. Verify a cluster manager is running.");
- }
-
+ InetSocketAddress socketAddress = null;
try {
// create a socket
- return SocketUtils.createSocket(service.getServiceAddress(), socketConfiguration);
+ socketAddress = getServiceAddress();
+ return SocketUtils.createSocket(socketAddress, socketConfiguration);
} catch (final IOException ioe) {
- throw new ProtocolException("Failed to create socket due to: " + ioe, ioe);
+ if (socketAddress == null) {
+ throw new ProtocolException("Failed to create socket due to: " + ioe, ioe);
+ } else {
+ throw new ProtocolException("Failed to create socket to " + socketAddress + " due to: " + ioe, ioe);
+ }
}
}
@@ -156,4 +140,6 @@ public class NodeProtocolSenderImpl implements NodeProtocolSender {
SocketUtils.closeQuietly(socket);
}
}
+
+ protected abstract InetSocketAddress getServiceAddress() throws IOException;
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterManagerProtocolSender.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterCoordinationProtocolSender.java
similarity index 84%
rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterManagerProtocolSender.java
rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterCoordinationProtocolSender.java
index 010aed7158..a1af0f8f5c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterManagerProtocolSender.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterCoordinationProtocolSender.java
@@ -19,8 +19,6 @@ package org.apache.nifi.cluster.protocol;
import java.util.Set;
import org.apache.nifi.cluster.protocol.message.DisconnectMessage;
-import org.apache.nifi.cluster.protocol.message.FlowRequestMessage;
-import org.apache.nifi.cluster.protocol.message.FlowResponseMessage;
import org.apache.nifi.cluster.protocol.message.NodeStatusChangeMessage;
import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage;
import org.apache.nifi.cluster.protocol.message.ReconnectionResponseMessage;
@@ -30,16 +28,7 @@ import org.apache.nifi.reporting.BulletinRepository;
* An interface for sending protocol messages from the cluster manager to nodes.
*
*/
-public interface ClusterManagerProtocolSender {
-
- /**
- * Sends a "flow request" message to a node.
- *
- * @param msg a message
- * @return the response
- * @throws ProtocolException if communication failed
- */
- FlowResponseMessage requestFlow(FlowRequestMessage msg) throws ProtocolException;
+public interface ClusterCoordinationProtocolSender {
/**
* Sends a "reconnection request" message to a node.
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ComponentRevision.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ComponentRevision.java
new file mode 100644
index 0000000000..5c5257a00f
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ComponentRevision.java
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+import org.apache.nifi.web.Revision;
+
+public class ComponentRevision {
+ private Long version;
+ private String clientId;
+ private String componentId;
+
+ public Long getVersion() {
+ return version;
+ }
+
+ public void setVersion(Long version) {
+ this.version = version;
+ }
+
+ public String getClientId() {
+ return clientId;
+ }
+
+ public void setClientId(String clientId) {
+ this.clientId = clientId;
+ }
+
+ public String getComponentId() {
+ return componentId;
+ }
+
+ public void setComponentId(String componentId) {
+ this.componentId = componentId;
+ }
+
+ public Revision toRevision() {
+ return new Revision(getVersion(), getClientId(), getComponentId());
+ }
+
+ public static ComponentRevision fromRevision(final Revision revision) {
+ final ComponentRevision componentRevision = new ComponentRevision();
+ componentRevision.setVersion(revision.getVersion());
+ componentRevision.setClientId(revision.getClientId());
+ componentRevision.setComponentId(revision.getComponentId());
+ return componentRevision;
+ }
+
+ @Override
+ public boolean equals(final Object obj) {
+ if (obj == null) {
+ return false;
+ }
+ if (obj == this) {
+ return true;
+ }
+
+ if ((obj instanceof ComponentRevision) == false) {
+ return false;
+ }
+
+ ComponentRevision thatRevision = (ComponentRevision) obj;
+ // ensure that component ID's are the same (including null)
+ if (thatRevision.getComponentId() == null && getComponentId() != null) {
+ return false;
+ }
+ if (thatRevision.getComponentId() != null && getComponentId() == null) {
+ return false;
+ }
+ if (thatRevision.getComponentId() != null && !thatRevision.getComponentId().equals(getComponentId())) {
+ return false;
+ }
+
+ if (this.version != null && this.version.equals(thatRevision.version)) {
+ return true;
+ } else {
+ return clientId != null && !clientId.trim().isEmpty() && clientId.equals(thatRevision.getClientId());
+ }
+
+ }
+
+ @Override
+ public int hashCode() {
+ int hash = 5;
+ hash = 59 * hash + (this.componentId != null ? this.componentId.hashCode() : 0);
+ hash = 59 * hash + (this.version != null ? this.version.hashCode() : 0);
+ hash = 59 * hash + (this.clientId != null ? this.clientId.hashCode() : 0);
+ return hash;
+ }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionResponse.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionResponse.java
index e6d8cf6f88..5aea3f1bc3 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionResponse.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionResponse.java
@@ -16,10 +16,14 @@
*/
package org.apache.nifi.cluster.protocol;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-
+import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus;
import org.apache.nifi.cluster.protocol.jaxb.message.ConnectionResponseAdapter;
+import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
/**
* The cluster manager's response to a node's connection request. If the manager
* has a current copy of the data flow, then it is returned with a node
@@ -34,15 +38,19 @@ public class ConnectionResponse {
private final String rejectionReason;
private final int tryLaterSeconds;
private final NodeIdentifier nodeIdentifier;
- private final StandardDataFlow dataFlow;
+ private final DataFlow dataFlow;
private final Integer managerRemoteInputPort;
private final Boolean managerRemoteCommsSecure;
private final String instanceId;
+ private final List nodeStatuses;
+ private final List componentRevisions;
- private volatile String clusterManagerDN;
+ private volatile String coordinatorDN;
+
+ public ConnectionResponse(final NodeIdentifier nodeIdentifier, final DataFlow dataFlow,
+ final Integer managerRemoteInputPort, final Boolean managerRemoteCommsSecure, final String instanceId,
+ final List nodeStatuses, final List componentRevisions) {
- public ConnectionResponse(final NodeIdentifier nodeIdentifier, final StandardDataFlow dataFlow,
- final Integer managerRemoteInputPort, final Boolean managerRemoteCommsSecure, final String instanceId) {
if (nodeIdentifier == null) {
throw new IllegalArgumentException("Node identifier may not be empty or null.");
} else if (dataFlow == null) {
@@ -55,6 +63,8 @@ public class ConnectionResponse {
this.managerRemoteInputPort = managerRemoteInputPort;
this.managerRemoteCommsSecure = managerRemoteCommsSecure;
this.instanceId = instanceId;
+ this.nodeStatuses = Collections.unmodifiableList(new ArrayList<>(nodeStatuses));
+ this.componentRevisions = componentRevisions == null ? Collections.emptyList() : Collections.unmodifiableList(new ArrayList<>(componentRevisions));
}
public ConnectionResponse(final int tryLaterSeconds) {
@@ -68,6 +78,8 @@ public class ConnectionResponse {
this.managerRemoteInputPort = null;
this.managerRemoteCommsSecure = null;
this.instanceId = null;
+ this.nodeStatuses = null;
+ this.componentRevisions = null;
}
private ConnectionResponse(final String rejectionReason) {
@@ -78,6 +90,8 @@ public class ConnectionResponse {
this.managerRemoteInputPort = null;
this.managerRemoteCommsSecure = null;
this.instanceId = null;
+ this.nodeStatuses = null;
+ this.componentRevisions = null;
}
public static ConnectionResponse createBlockedByFirewallResponse() {
@@ -104,7 +118,7 @@ public class ConnectionResponse {
return tryLaterSeconds;
}
- public StandardDataFlow getDataFlow() {
+ public DataFlow getDataFlow() {
return dataFlow;
}
@@ -124,15 +138,23 @@ public class ConnectionResponse {
return instanceId;
}
- public void setClusterManagerDN(final String dn) {
- this.clusterManagerDN = dn;
+ public void setCoordinatorDN(final String dn) {
+ this.coordinatorDN = dn;
+ }
+
+ public List getNodeConnectionStatuses() {
+ return nodeStatuses;
+ }
+
+ public List getComponentRevisions() {
+ return componentRevisions;
}
/**
- * @return the DN of the NCM, if it is available or null
+ * @return the DN of the Coordinator, if it is available or null
* otherwise
*/
- public String getClusterManagerDN() {
- return clusterManagerDN;
+ public String getCoordinatorDN() {
+ return coordinatorDN;
}
}
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 f2b0fde9ec..2135f20dc0 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,7 +16,11 @@
*/
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;
@@ -31,17 +35,17 @@ import org.apache.nifi.cluster.protocol.jaxb.message.HeartbeatAdapter;
public class Heartbeat {
private final NodeIdentifier nodeIdentifier;
- private final boolean primary;
+ private final Set roles;
private final NodeConnectionStatus connectionStatus;
private final long createdTimestamp;
private final byte[] payload;
- public Heartbeat(final NodeIdentifier nodeIdentifier, final boolean primary, final NodeConnectionStatus connectionStatus, final byte[] payload) {
+ public Heartbeat(final NodeIdentifier nodeIdentifier, final Set roles, final NodeConnectionStatus connectionStatus, final byte[] payload) {
if (nodeIdentifier == null) {
throw new IllegalArgumentException("Node Identifier may not be null.");
}
this.nodeIdentifier = nodeIdentifier;
- this.primary = primary;
+ this.roles = roles == null ? Collections.emptySet() : Collections.unmodifiableSet(new HashSet<>(roles));
this.connectionStatus = connectionStatus;
this.payload = payload;
this.createdTimestamp = new Date().getTime();
@@ -55,8 +59,8 @@ public class Heartbeat {
return payload;
}
- public boolean isPrimary() {
- return primary;
+ public Set getRoles() {
+ return roles;
}
public NodeConnectionStatus getConnectionStatus() {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderListener.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterCoordinationProtocolSenderListener.java
similarity index 84%
rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderListener.java
rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterCoordinationProtocolSenderListener.java
index 2fc05b01a3..e97712a726 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderListener.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterCoordinationProtocolSenderListener.java
@@ -20,14 +20,12 @@ import java.io.IOException;
import java.util.Collection;
import java.util.Set;
-import org.apache.nifi.cluster.protocol.ClusterManagerProtocolSender;
+import org.apache.nifi.cluster.protocol.ClusterCoordinationProtocolSender;
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.DisconnectMessage;
-import org.apache.nifi.cluster.protocol.message.FlowRequestMessage;
-import org.apache.nifi.cluster.protocol.message.FlowResponseMessage;
import org.apache.nifi.cluster.protocol.message.NodeStatusChangeMessage;
import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage;
import org.apache.nifi.cluster.protocol.message.ReconnectionResponseMessage;
@@ -38,13 +36,13 @@ import org.apache.nifi.reporting.BulletinRepository;
* cluster manager.
*
*/
-public class ClusterManagerProtocolSenderListener implements ClusterManagerProtocolSender, ProtocolListener {
+public class ClusterCoordinationProtocolSenderListener implements ClusterCoordinationProtocolSender, ProtocolListener {
- private final ClusterManagerProtocolSender sender;
+ private final ClusterCoordinationProtocolSender sender;
private final ProtocolListener listener;
- public ClusterManagerProtocolSenderListener(final ClusterManagerProtocolSender sender, final ProtocolListener listener) {
+ public ClusterCoordinationProtocolSenderListener(final ClusterCoordinationProtocolSender sender, final ProtocolListener listener) {
if (sender == null) {
throw new IllegalArgumentException("ClusterManagerProtocolSender may not be null.");
} else if (listener == null) {
@@ -96,11 +94,6 @@ public class ClusterManagerProtocolSenderListener implements ClusterManagerProto
sender.setBulletinRepository(bulletinRepository);
}
- @Override
- public FlowResponseMessage requestFlow(final FlowRequestMessage msg) throws ProtocolException {
- return sender.requestFlow(msg);
- }
-
@Override
public ReconnectionResponseMessage requestReconnection(final ReconnectionRequestMessage msg) throws ProtocolException {
return sender.requestReconnection(msg);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscovery.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscovery.java
deleted file mode 100644
index f808c83ab7..0000000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscovery.java
+++ /dev/null
@@ -1,180 +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.protocol.impl;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.Collection;
-import java.util.Collections;
-import org.apache.nifi.io.socket.multicast.DiscoverableService;
-import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl;
-import org.apache.nifi.io.socket.multicast.MulticastConfiguration;
-import org.apache.nifi.io.socket.multicast.MulticastServiceDiscovery;
-import org.apache.nifi.reporting.BulletinRepository;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-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.ProtocolMessage;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType;
-import org.apache.nifi.cluster.protocol.message.ServiceBroadcastMessage;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * An implementation for discovering services by way of "service broadcast" type
- * protocol messages over multicast.
- *
- * The client caller is responsible for starting and stopping the service
- * discovery. The instance must be stopped before termination of the JVM to
- * ensure proper resource clean-up.
- *
- */
-public class ClusterServiceDiscovery implements MulticastServiceDiscovery, ProtocolListener {
-
- private static final Logger logger = LoggerFactory.getLogger(ClusterServiceDiscovery.class);
- private final String serviceName;
- private final MulticastConfiguration multicastConfiguration;
- private final MulticastProtocolListener listener;
- private volatile BulletinRepository bulletinRepository;
-
- /*
- * guarded by this
- */
- private DiscoverableService service;
-
- public ClusterServiceDiscovery(final String serviceName, final InetSocketAddress multicastAddress,
- final MulticastConfiguration multicastConfiguration, final ProtocolContext protocolContext) {
-
- if (StringUtils.isBlank(serviceName)) {
- throw new IllegalArgumentException("Service name may not be null or empty.");
- } else if (multicastAddress == null) {
- throw new IllegalArgumentException("Multicast address may not be null.");
- } else if (multicastAddress.getAddress().isMulticastAddress() == false) {
- throw new IllegalArgumentException("Multicast group must be a Class D address.");
- } else if (protocolContext == null) {
- throw new IllegalArgumentException("Protocol Context may not be null.");
- } else if (multicastConfiguration == null) {
- throw new IllegalArgumentException("Multicast configuration may not be null.");
- }
-
- this.serviceName = serviceName;
- this.multicastConfiguration = multicastConfiguration;
- this.listener = new MulticastProtocolListener(1, multicastAddress, multicastConfiguration, protocolContext);
- listener.addHandler(new ClusterManagerServiceBroadcastHandler());
- }
-
- @Override
- public void setBulletinRepository(final BulletinRepository bulletinRepository) {
- this.bulletinRepository = bulletinRepository;
- }
-
- @Override
- public synchronized DiscoverableService getService() {
- return service;
- }
-
- @Override
- public InetSocketAddress getMulticastAddress() {
- return listener.getMulticastAddress();
- }
-
- @Override
- public Collection getHandlers() {
- return Collections.unmodifiableCollection(listener.getHandlers());
- }
-
- @Override
- public void addHandler(ProtocolHandler handler) {
- listener.addHandler(handler);
- }
-
- @Override
- public boolean removeHandler(ProtocolHandler handler) {
- return listener.removeHandler(handler);
- }
-
- @Override
- public boolean isRunning() {
- return listener.isRunning();
- }
-
- @Override
- public void start() throws IOException {
- if (isRunning()) {
- throw new IllegalStateException("Instance is already running.");
- }
- listener.start();
- }
-
- @Override
- public void stop() throws IOException {
- if (isRunning() == false) {
- throw new IllegalStateException("Instance is already stopped.");
- }
- listener.stop();
- }
-
- public String getServiceName() {
- return serviceName;
- }
-
- public MulticastConfiguration getMulticastConfiguration() {
- return multicastConfiguration;
- }
-
- private class ClusterManagerServiceBroadcastHandler implements ProtocolHandler {
-
- @Override
- public boolean canHandle(final ProtocolMessage msg) {
- return MessageType.SERVICE_BROADCAST == msg.getType();
- }
-
- @Override
- public ProtocolMessage handle(final ProtocolMessage msg) throws ProtocolException {
- synchronized (ClusterServiceDiscovery.this) {
- if (canHandle(msg) == false) {
- throw new ProtocolException("Handler cannot handle message type: " + msg.getType());
- } else {
- final ServiceBroadcastMessage broadcastMsg = (ServiceBroadcastMessage) msg;
- if (serviceName.equals(broadcastMsg.getServiceName())) {
- final DiscoverableService oldService = service;
- if (oldService == null
- || broadcastMsg.getAddress().equalsIgnoreCase(oldService.getServiceAddress().getHostName()) == false
- || broadcastMsg.getPort() != oldService.getServiceAddress().getPort()) {
- service = new DiscoverableServiceImpl(serviceName, InetSocketAddress.createUnresolved(broadcastMsg.getAddress(), broadcastMsg.getPort()));
- final InetSocketAddress oldServiceAddress = (oldService == null) ? null : oldService.getServiceAddress();
- logger.info(String.format("Updating cluster service address for '%s' from '%s' to '%s'",
- serviceName, prettyPrint(oldServiceAddress), prettyPrint(service.getServiceAddress())));
- }
- }
- return null;
- }
- }
- }
- }
-
- private String prettyPrint(final InetSocketAddress address) {
- if (address == null) {
- return "0.0.0.0:0";
- } else {
- return address.getHostName() + ":" + address.getPort();
- }
- }
-}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocator.java
deleted file mode 100644
index a49847f89d..0000000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocator.java
+++ /dev/null
@@ -1,228 +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.protocol.impl;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import org.apache.nifi.io.socket.multicast.DiscoverableService;
-import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl;
-import org.apache.nifi.io.socket.multicast.ServiceDiscovery;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Implements the ServiceLocator interface for locating the socket address of a
- * cluster service. Depending on configuration, the address may be located using
- * service discovery. If using service discovery, then the service methods must
- * be used for starting and stopping discovery.
- *
- * Service discovery may be used in conjunction with a fixed port. In this case,
- * the service discovery will yield the service IP/host while the fixed port
- * will be used for the port.
- *
- * Alternatively, the instance may be configured with exact service location, in
- * which case, no service discovery occurs and the caller will always receive
- * the configured service.
- *
- */
-public class ClusterServiceLocator implements ServiceDiscovery {
-
- private static final Logger logger = LoggerFactory.getLogger(ClusterServiceLocator.class);
-
- private final String serviceName;
-
- private final ClusterServiceDiscovery serviceDiscovery;
-
- private final DiscoverableService fixedService;
-
- private final int fixedServicePort;
-
- private final AttemptsConfig attemptsConfig = new AttemptsConfig();
-
- private final AtomicBoolean running = new AtomicBoolean(false);
-
- public ClusterServiceLocator(final ClusterServiceDiscovery serviceDiscovery) {
- if (serviceDiscovery == null) {
- throw new IllegalArgumentException("ClusterServiceDiscovery may not be null.");
- }
- this.serviceDiscovery = serviceDiscovery;
- this.fixedService = null;
- this.fixedServicePort = 0;
- this.serviceName = serviceDiscovery.getServiceName();
- }
-
- public ClusterServiceLocator(final ClusterServiceDiscovery serviceDiscovery, final int fixedServicePort) {
- if (serviceDiscovery == null) {
- throw new IllegalArgumentException("ClusterServiceDiscovery may not be null.");
- }
- this.serviceDiscovery = serviceDiscovery;
- this.fixedService = null;
- this.fixedServicePort = fixedServicePort;
- this.serviceName = serviceDiscovery.getServiceName();
- }
-
- public ClusterServiceLocator(final DiscoverableService fixedService) {
- if (fixedService == null) {
- throw new IllegalArgumentException("Service may not be null.");
- }
- this.serviceDiscovery = null;
- this.fixedService = fixedService;
- this.fixedServicePort = 0;
- this.serviceName = fixedService.getServiceName();
- }
-
- @Override
- public DiscoverableService getService() {
-
- final int numAttemptsValue;
- final int secondsBetweenAttempts;
- synchronized (this) {
- numAttemptsValue = attemptsConfig.numAttempts;
- secondsBetweenAttempts = attemptsConfig.getTimeBetweenAttempts();
- }
-
- // try for a configured amount of attempts to retrieve the service address
- for (int i = 0; i < numAttemptsValue; i++) {
-
- if (fixedService != null) {
- return fixedService;
- } else if (serviceDiscovery != null) {
-
- final DiscoverableService discoveredService = serviceDiscovery.getService();
-
- // if we received an address
- if (discoveredService != null) {
- // if we were configured with a fixed port, then use the discovered host and fixed port; otherwise use the discovered address
- if (fixedServicePort > 0) {
- // create service using discovered service name and address with fixed service port
- final InetSocketAddress addr = InetSocketAddress.createUnresolved(discoveredService.getServiceAddress().getHostName(), fixedServicePort);
- final DiscoverableService result = new DiscoverableServiceImpl(discoveredService.getServiceName(), addr);
- return result;
- } else {
- return discoveredService;
- }
- }
- }
-
- // could not obtain service address, so sleep a bit
- try {
- logger.debug(String.format("Locating Cluster Service '%s' Attempt: %d of %d failed. Trying again in %d seconds.",
- serviceName, (i + 1), numAttemptsValue, secondsBetweenAttempts));
- Thread.sleep(secondsBetweenAttempts * 1000);
- } catch (final InterruptedException ie) {
- break;
- }
-
- }
-
- return null;
- }
-
- public boolean isRunning() {
- if (serviceDiscovery != null) {
- return serviceDiscovery.isRunning();
- } else {
- return running.get();
- }
- }
-
- public void start() throws IOException {
-
- if (isRunning()) {
- throw new IllegalStateException("Instance is already started.");
- }
-
- if (serviceDiscovery != null) {
- serviceDiscovery.start();
- }
- running.set(true);
- }
-
- public void stop() throws IOException {
-
- if (isRunning() == false) {
- throw new IllegalStateException("Instance is already stopped.");
- }
-
- if (serviceDiscovery != null) {
- serviceDiscovery.stop();
- }
- running.set(false);
- }
-
- public synchronized void setAttemptsConfig(final AttemptsConfig config) {
- if (config == null) {
- throw new IllegalArgumentException("Attempts configuration may not be null.");
- }
- this.attemptsConfig.numAttempts = config.numAttempts;
- this.attemptsConfig.timeBetweenAttempts = config.timeBetweenAttempts;
- this.attemptsConfig.timeBetweenAttempsUnit = config.timeBetweenAttempsUnit;
- }
-
- public synchronized AttemptsConfig getAttemptsConfig() {
- final AttemptsConfig config = new AttemptsConfig();
- config.numAttempts = this.attemptsConfig.numAttempts;
- config.timeBetweenAttempts = this.attemptsConfig.timeBetweenAttempts;
- config.timeBetweenAttempsUnit = this.attemptsConfig.timeBetweenAttempsUnit;
- return config;
- }
-
- public static class AttemptsConfig {
-
- private int numAttempts = 1;
-
- private int timeBetweenAttempts = 1;
-
- private TimeUnit timeBetweenAttempsUnit = TimeUnit.SECONDS;
-
- public int getNumAttempts() {
- return numAttempts;
- }
-
- public void setNumAttempts(int numAttempts) {
- if (numAttempts <= 0) {
- throw new IllegalArgumentException("Number of attempts must be positive: " + numAttempts);
- }
- this.numAttempts = numAttempts;
- }
-
- public TimeUnit getTimeBetweenAttemptsUnit() {
- return timeBetweenAttempsUnit;
- }
-
- public void setTimeBetweenAttempsUnit(TimeUnit timeBetweenAttempsUnit) {
- if (timeBetweenAttempts <= 0) {
- throw new IllegalArgumentException("Time between attempts must be positive: " + numAttempts);
- }
- this.timeBetweenAttempsUnit = timeBetweenAttempsUnit;
- }
-
- public int getTimeBetweenAttempts() {
- return timeBetweenAttempts;
- }
-
- public void setTimeBetweenAttempts(int timeBetweenAttempts) {
- if (timeBetweenAttempts <= 0) {
- throw new IllegalArgumentException("Time between attempts must be positive: " + numAttempts);
- }
- this.timeBetweenAttempts = timeBetweenAttempts;
- }
-
- }
-}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcaster.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcaster.java
deleted file mode 100644
index 0bb13d40af..0000000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcaster.java
+++ /dev/null
@@ -1,185 +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.protocol.impl;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.net.DatagramPacket;
-import java.net.InetSocketAddress;
-import java.net.MulticastSocket;
-import java.util.Collections;
-import java.util.Set;
-import java.util.Timer;
-import java.util.TimerTask;
-import java.util.concurrent.CopyOnWriteArraySet;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.nifi.io.socket.multicast.DiscoverableService;
-import org.apache.nifi.io.socket.multicast.MulticastConfiguration;
-import org.apache.nifi.io.socket.multicast.MulticastServicesBroadcaster;
-import org.apache.nifi.io.socket.multicast.MulticastUtils;
-import org.apache.nifi.logging.NiFiLog;
-import org.apache.nifi.util.FormatUtils;
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-import org.apache.nifi.cluster.protocol.message.ServiceBroadcastMessage;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Broadcasts services used by the clustering software using multicast
- * communication. A configurable delay occurs after broadcasting the collection
- * of services.
- *
- * The client caller is responsible for starting and stopping the broadcasting.
- * The instance must be stopped before termination of the JVM to ensure proper
- * resource clean-up.
- *
- */
-public class ClusterServicesBroadcaster implements MulticastServicesBroadcaster {
-
- private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(ClusterServicesBroadcaster.class));
-
- private final Set services = new CopyOnWriteArraySet<>();
-
- private final InetSocketAddress multicastAddress;
-
- private final MulticastConfiguration multicastConfiguration;
-
- private final ProtocolContext protocolContext;
-
- private final int broadcastDelayMs;
-
- private Timer broadcaster;
-
- private MulticastSocket multicastSocket;
-
- public ClusterServicesBroadcaster(final InetSocketAddress multicastAddress,
- final MulticastConfiguration multicastConfiguration,
- final ProtocolContext protocolContext, final String broadcastDelay) {
-
- if (multicastAddress == null) {
- throw new IllegalArgumentException("Multicast address may not be null.");
- } else if (multicastAddress.getAddress().isMulticastAddress() == false) {
- throw new IllegalArgumentException("Multicast group address is not a Class D IP address.");
- } else if (protocolContext == null) {
- throw new IllegalArgumentException("Protocol Context may not be null.");
- } else if (multicastConfiguration == null) {
- throw new IllegalArgumentException("Multicast configuration may not be null.");
- }
-
- this.services.addAll(services);
- this.multicastAddress = multicastAddress;
- this.multicastConfiguration = multicastConfiguration;
- this.protocolContext = protocolContext;
- this.broadcastDelayMs = (int) FormatUtils.getTimeDuration(broadcastDelay, TimeUnit.MILLISECONDS);
- }
-
- public void start() throws IOException {
-
- if (isRunning()) {
- throw new IllegalStateException("Instance is already started.");
- }
-
- // setup socket
- multicastSocket = MulticastUtils.createMulticastSocket(multicastConfiguration);
-
- // setup broadcaster
- broadcaster = new Timer("Cluster Services Broadcaster", /* is daemon */ true);
- broadcaster.schedule(new TimerTask() {
- @Override
- public void run() {
- for (final DiscoverableService service : services) {
- try {
-
- final InetSocketAddress serviceAddress = service.getServiceAddress();
- logger.debug(String.format("Broadcasting Cluster Service '%s' at address %s:%d",
- service.getServiceName(), serviceAddress.getHostName(), serviceAddress.getPort()));
-
- // create message
- final ServiceBroadcastMessage msg = new ServiceBroadcastMessage();
- msg.setServiceName(service.getServiceName());
- msg.setAddress(serviceAddress.getHostName());
- msg.setPort(serviceAddress.getPort());
-
- // marshal message to output stream
- final ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller();
- final ByteArrayOutputStream baos = new ByteArrayOutputStream();
- marshaller.marshal(msg, baos);
- final byte[] packetBytes = baos.toByteArray();
-
- // send message
- final DatagramPacket packet = new DatagramPacket(packetBytes, packetBytes.length, multicastAddress);
- multicastSocket.send(packet);
-
- } catch (final Exception ex) {
- logger.warn(String.format("Cluster Services Broadcaster failed broadcasting service '%s' due to: %s", service.getServiceName(), ex), ex);
- }
- }
- }
- }, 0, broadcastDelayMs);
- }
-
- public boolean isRunning() {
- return (broadcaster != null);
- }
-
- public void stop() {
-
- if (isRunning() == false) {
- throw new IllegalStateException("Instance is already stopped.");
- }
-
- broadcaster.cancel();
- broadcaster = null;
-
- // close socket
- MulticastUtils.closeQuietly(multicastSocket);
-
- }
-
- @Override
- public int getBroadcastDelayMs() {
- return broadcastDelayMs;
- }
-
- @Override
- public Set getServices() {
- return Collections.unmodifiableSet(services);
- }
-
- @Override
- public InetSocketAddress getMulticastAddress() {
- return multicastAddress;
- }
-
- @Override
- public boolean addService(final DiscoverableService service) {
- return services.add(service);
- }
-
- @Override
- public boolean removeService(final String serviceName) {
- for (final DiscoverableService service : services) {
- if (service.getServiceName().equals(serviceName)) {
- return services.remove(service);
- }
- }
- return false;
- }
-}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListener.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListener.java
deleted file mode 100644
index 8ca51419fa..0000000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListener.java
+++ /dev/null
@@ -1,203 +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.protocol.impl;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.net.DatagramPacket;
-import java.net.InetSocketAddress;
-import java.net.MulticastSocket;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.UUID;
-import java.util.concurrent.CopyOnWriteArrayList;
-import org.apache.nifi.io.socket.multicast.MulticastConfiguration;
-import org.apache.nifi.io.socket.multicast.MulticastListener;
-import org.apache.nifi.logging.NiFiLog;
-import org.apache.nifi.reporting.Bulletin;
-import org.apache.nifi.reporting.BulletinRepository;
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-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.ProtocolMessageMarshaller;
-import org.apache.nifi.cluster.protocol.ProtocolMessageUnmarshaller;
-import org.apache.nifi.cluster.protocol.message.MulticastProtocolMessage;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-import org.apache.nifi.events.BulletinFactory;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Implements a listener for protocol messages sent over multicast. If a message
- * is of type MulticastProtocolMessage, then the underlying protocol message is
- * passed to the handler. If the receiving handler produces a message response,
- * then the message is wrapped with a MulticastProtocolMessage before being sent
- * to the originator.
- *
- * The client caller is responsible for starting and stopping the listener. The
- * instance must be stopped before termination of the JVM to ensure proper
- * resource clean-up.
- *
- */
-public class MulticastProtocolListener extends MulticastListener implements ProtocolListener {
-
- private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(MulticastProtocolListener.class));
-
- // immutable members
- private final Collection handlers = new CopyOnWriteArrayList<>();
- private final String listenerId = UUID.randomUUID().toString();
- private final ProtocolContext protocolContext;
- private volatile BulletinRepository bulletinRepository;
-
- public MulticastProtocolListener(
- final int numThreads,
- final InetSocketAddress multicastAddress,
- final MulticastConfiguration configuration,
- final ProtocolContext protocolContext) {
-
- super(numThreads, multicastAddress, configuration);
-
- if (protocolContext == null) {
- throw new IllegalArgumentException("Protocol Context may not be null.");
- }
- this.protocolContext = protocolContext;
- }
-
- @Override
- public void setBulletinRepository(final BulletinRepository bulletinRepository) {
- this.bulletinRepository = bulletinRepository;
- }
-
- @Override
- public void start() throws IOException {
-
- if (super.isRunning()) {
- throw new IllegalStateException("Instance is already started.");
- }
-
- super.start();
-
- }
-
- @Override
- public void stop() throws IOException {
-
- if (super.isRunning() == false) {
- throw new IllegalStateException("Instance is already stopped.");
- }
-
- // shutdown listener
- super.stop();
-
- }
-
- @Override
- public Collection getHandlers() {
- return Collections.unmodifiableCollection(handlers);
- }
-
- @Override
- public void addHandler(final ProtocolHandler handler) {
- if (handler == null) {
- throw new NullPointerException("Protocol handler may not be null.");
- }
- handlers.add(handler);
- }
-
- @Override
- public boolean removeHandler(final ProtocolHandler handler) {
- return handlers.remove(handler);
- }
-
- @Override
- public void dispatchRequest(final MulticastSocket multicastSocket, final DatagramPacket packet) {
-
- try {
-
- // unmarshall message
- final ProtocolMessageUnmarshaller unmarshaller = protocolContext.createUnmarshaller();
- final ProtocolMessage request = unmarshaller.unmarshal(new ByteArrayInputStream(packet.getData(), 0, packet.getLength()));
-
- // unwrap multicast message, if necessary
- final ProtocolMessage unwrappedRequest;
- if (request instanceof MulticastProtocolMessage) {
- final MulticastProtocolMessage multicastRequest = (MulticastProtocolMessage) request;
- // don't process a message we sent
- if (listenerId.equals(multicastRequest.getId())) {
- return;
- } else {
- unwrappedRequest = multicastRequest.getProtocolMessage();
- }
- } else {
- unwrappedRequest = request;
- }
-
- // dispatch message to handler
- ProtocolHandler desiredHandler = null;
- for (final ProtocolHandler handler : getHandlers()) {
- if (handler.canHandle(unwrappedRequest)) {
- desiredHandler = handler;
- break;
- }
- }
-
- // if no handler found, throw exception; otherwise handle request
- if (desiredHandler == null) {
- throw new ProtocolException("No handler assigned to handle message type: " + request.getType());
- } else {
- final ProtocolMessage response = desiredHandler.handle(request);
- if (response != null) {
- try {
-
- // wrap with listener id
- final MulticastProtocolMessage multicastResponse = new MulticastProtocolMessage(listenerId, response);
-
- // marshal message
- final ByteArrayOutputStream baos = new ByteArrayOutputStream();
- final ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller();
- marshaller.marshal(multicastResponse, baos);
- final byte[] responseBytes = baos.toByteArray();
-
- final int maxPacketSizeBytes = getMaxPacketSizeBytes();
- if (responseBytes.length > maxPacketSizeBytes) {
- logger.warn("Cluster protocol handler '" + desiredHandler.getClass()
- + "' produced a multicast response with length greater than configured max packet size '" + maxPacketSizeBytes + "'");
- }
-
- // create and send packet
- final DatagramPacket responseDatagram = new DatagramPacket(responseBytes, responseBytes.length, getMulticastAddress().getAddress(), getMulticastAddress().getPort());
- multicastSocket.send(responseDatagram);
-
- } catch (final IOException ioe) {
- throw new ProtocolException("Failed marshalling protocol message in response to message type: " + request.getType() + " due to: " + ioe, ioe);
- }
- }
- }
-
- } catch (final Throwable t) {
- logger.warn("Failed processing protocol message due to " + t, t);
-
- if (bulletinRepository != null) {
- final Bulletin bulletin = BulletinFactory.createBulletin("Clustering", "WARNING", "Failed to process Protocol Message due to " + t.toString());
- bulletinRepository.addBulletin(bulletin);
- }
- }
- }
-}
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 3d0eb8e338..cc403310a9 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
@@ -30,7 +30,6 @@ import org.apache.nifi.cluster.protocol.message.HeartbeatMessage;
import org.apache.nifi.reporting.BulletinRepository;
public class NodeProtocolSenderListener implements NodeProtocolSender, ProtocolListener {
-
private final NodeProtocolSender sender;
private final ProtocolListener listener;
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListener.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListener.java
index 1345df318a..fd93b24405 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListener.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListener.java
@@ -25,6 +25,7 @@ import java.util.Collections;
import java.util.UUID;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.TimeUnit;
+
import org.apache.nifi.cluster.protocol.ProtocolContext;
import org.apache.nifi.cluster.protocol.ProtocolException;
import org.apache.nifi.cluster.protocol.ProtocolHandler;
@@ -35,7 +36,6 @@ import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
import org.apache.nifi.events.BulletinFactory;
import org.apache.nifi.io.socket.ServerSocketConfiguration;
import org.apache.nifi.io.socket.SocketListener;
-import org.apache.nifi.logging.NiFiLog;
import org.apache.nifi.reporting.Bulletin;
import org.apache.nifi.reporting.BulletinRepository;
import org.apache.nifi.security.util.CertificateUtils;
@@ -49,7 +49,7 @@ import org.slf4j.LoggerFactory;
*/
public class SocketProtocolListener extends SocketListener implements ProtocolListener {
- private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(SocketProtocolListener.class));
+ private static final Logger logger = LoggerFactory.getLogger(SocketProtocolListener.class);
private final ProtocolContext protocolContext;
private final Collection handlers = new CopyOnWriteArrayList<>();
private volatile BulletinRepository bulletinRepository;
@@ -122,7 +122,7 @@ public class SocketProtocolListener extends SocketListener implements ProtocolLi
final StopWatch stopWatch = new StopWatch(true);
hostname = socket.getInetAddress().getHostName();
final String requestId = UUID.randomUUID().toString();
- logger.info("Received request {} from {}", requestId, hostname);
+ logger.debug("Received request {} from {}", requestId, hostname);
String requestorDn = getRequestorDN(socket);
@@ -130,20 +130,23 @@ public class SocketProtocolListener extends SocketListener implements ProtocolLi
final ProtocolMessageUnmarshaller unmarshaller = protocolContext.createUnmarshaller();
final InputStream inStream = socket.getInputStream();
final CopyingInputStream copyingInputStream = new CopyingInputStream(inStream, maxMsgBuffer); // don't copy more than 1 MB
- logger.debug("Request {} has a message length of {}", requestId, copyingInputStream.getNumberOfBytesCopied());
final ProtocolMessage request;
try {
request = unmarshaller.unmarshal(copyingInputStream);
} finally {
receivedMessage = copyingInputStream.getBytesRead();
+ if (logger.isDebugEnabled()) {
+ logger.debug("Received message: " + new String(receivedMessage));
+ }
}
request.setRequestorDN(requestorDn);
// dispatch message to handler
ProtocolHandler desiredHandler = null;
- for (final ProtocolHandler handler : getHandlers()) {
+ final Collection handlers = getHandlers();
+ for (final ProtocolHandler handler : handlers) {
if (handler.canHandle(request)) {
desiredHandler = handler;
break;
@@ -152,6 +155,7 @@ public class SocketProtocolListener extends SocketListener implements ProtocolLi
// if no handler found, throw exception; otherwise handle request
if (desiredHandler == null) {
+ logger.error("Received request of type {} but none of the following Protocol Handlers were able to process the request: {}", request.getType(), handlers);
throw new ProtocolException("No handler assigned to handle message type: " + request.getType());
} else {
final ProtocolMessage response = desiredHandler.handle(request);
@@ -169,7 +173,8 @@ public class SocketProtocolListener extends SocketListener implements ProtocolLi
}
stopWatch.stop();
- logger.info("Finished processing request {} (type={}, length={} bytes) in {} millis", requestId, request.getType(), receivedMessage.length, stopWatch.getDuration(TimeUnit.MILLISECONDS));
+ logger.info("Finished processing request {} (type={}, length={} bytes) from {} in {} millis",
+ requestId, request.getType(), receivedMessage.length, hostname, stopWatch.getDuration(TimeUnit.MILLISECONDS));
} catch (final IOException | ProtocolException e) {
logger.warn("Failed processing protocol message from " + hostname + " due to " + e, e);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImpl.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/StandardClusterCoordinationProtocolSender.java
similarity index 80%
rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImpl.java
rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/StandardClusterCoordinationProtocolSender.java
index 63ba5ba66c..4b58886ea6 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImpl.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/StandardClusterCoordinationProtocolSender.java
@@ -24,17 +24,17 @@ import java.net.SocketException;
import java.util.Set;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.nifi.cluster.protocol.ClusterManagerProtocolSender;
+import org.apache.nifi.cluster.protocol.ClusterCoordinationProtocolSender;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.cluster.protocol.ProtocolContext;
import org.apache.nifi.cluster.protocol.ProtocolException;
import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller;
import org.apache.nifi.cluster.protocol.ProtocolMessageUnmarshaller;
import org.apache.nifi.cluster.protocol.message.DisconnectMessage;
-import org.apache.nifi.cluster.protocol.message.FlowRequestMessage;
-import org.apache.nifi.cluster.protocol.message.FlowResponseMessage;
import org.apache.nifi.cluster.protocol.message.NodeStatusChangeMessage;
import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType;
@@ -56,13 +56,13 @@ import org.apache.nifi.util.NiFiProperties;
* allowable time for communication with the node.
*
*/
-public class ClusterManagerProtocolSenderImpl implements ClusterManagerProtocolSender {
+public class StandardClusterCoordinationProtocolSender implements ClusterCoordinationProtocolSender {
private final ProtocolContext protocolContext;
private final SocketConfiguration socketConfiguration;
private int handshakeTimeoutSeconds;
- public ClusterManagerProtocolSenderImpl(final SocketConfiguration socketConfiguration, final ProtocolContext protocolContext) {
+ public StandardClusterCoordinationProtocolSender(final SocketConfiguration socketConfiguration, final ProtocolContext protocolContext) {
if (socketConfiguration == null) {
throw new IllegalArgumentException("Socket configuration may not be null.");
} else if (protocolContext == null) {
@@ -77,47 +77,6 @@ public class ClusterManagerProtocolSenderImpl implements ClusterManagerProtocolS
public void setBulletinRepository(final BulletinRepository bulletinRepository) {
}
- /**
- * Requests the data flow from a node.
- *
- * @param msg a message
- * @return the message response
- * @throws ProtocolException if the message failed to be sent or the
- * response was malformed
- */
- @Override
- public FlowResponseMessage requestFlow(final FlowRequestMessage msg) throws ProtocolException {
- Socket socket = null;
- try {
- socket = createSocket(msg.getNodeId(), false);
-
- try {
- // marshal message to output stream
- final ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller();
- marshaller.marshal(msg, socket.getOutputStream());
- } 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.FLOW_RESPONSE + "' protocol message due to: " + ioe, ioe);
- }
-
- if (MessageType.FLOW_RESPONSE == response.getType()) {
- return (FlowResponseMessage) response;
- } else {
- throw new ProtocolException("Expected message type '" + MessageType.FLOW_RESPONSE + "' but found '" + response.getType() + "'");
- }
-
- } finally {
- SocketUtils.closeQuietly(socket);
- }
- }
/**
* Requests a node to reconnect to the cluster. The configured value for
@@ -225,8 +184,12 @@ public class ClusterManagerProtocolSenderImpl implements ClusterManagerProtocolS
@Override
public void notifyNodeStatusChange(final Set nodesToNotify, final NodeStatusChangeMessage msg) {
+ if (nodesToNotify.isEmpty()) {
+ return;
+ }
+
final NiFiProperties properties = NiFiProperties.getInstance();
- final int numThreads = Math.min(nodesToNotify.size(), properties.getClusterManagerProtocolThreads());
+ final int numThreads = Math.min(nodesToNotify.size(), properties.getClusterNodeProtocolThreads());
final byte[] msgBytes;
try (final ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
@@ -237,7 +200,18 @@ public class ClusterManagerProtocolSenderImpl implements ClusterManagerProtocolS
throw new ProtocolException("Failed to marshal NodeStatusChangeMessage", e);
}
- final ExecutorService executor = Executors.newFixedThreadPool(numThreads);
+ final ExecutorService executor = Executors.newFixedThreadPool(numThreads, new ThreadFactory() {
+ private final AtomicInteger counter = new AtomicInteger(0);
+
+ @Override
+ public Thread newThread(final Runnable r) {
+ final Thread thread = Executors.defaultThreadFactory().newThread(r);
+ thread.setDaemon(true);
+ thread.setName("Notify Cluster of Node Status Change-" + counter.incrementAndGet());
+ return thread;
+ }
+ });
+
for (final NodeIdentifier nodeId : nodesToNotify) {
executor.submit(new Runnable() {
@Override
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/AdaptedConnectionResponse.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionResponse.java
index d42515e4ea..a4eb46e886 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionResponse.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionResponse.java
@@ -16,32 +16,38 @@
*/
package org.apache.nifi.cluster.protocol.jaxb.message;
+import java.util.List;
+
import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
+import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus;
+import org.apache.nifi.cluster.protocol.ComponentRevision;
+import org.apache.nifi.cluster.protocol.DataFlow;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.apache.nifi.cluster.protocol.StandardDataFlow;
/**
*/
public class AdaptedConnectionResponse {
- private StandardDataFlow dataFlow;
+ private DataFlow dataFlow;
private NodeIdentifier nodeIdentifier;
private String rejectionReason;
private int tryLaterSeconds;
private Integer managerRemoteInputPort;
private Boolean managerRemoteCommsSecure;
private String instanceId;
+ private List nodeStatuses;
+ private List componentRevisions;
public AdaptedConnectionResponse() {
}
@XmlJavaTypeAdapter(DataFlowAdapter.class)
- public StandardDataFlow getDataFlow() {
+ public DataFlow getDataFlow() {
return dataFlow;
}
- public void setDataFlow(StandardDataFlow dataFlow) {
+ public void setDataFlow(DataFlow dataFlow) {
this.dataFlow = dataFlow;
}
@@ -97,4 +103,20 @@ public class AdaptedConnectionResponse {
public String getInstanceId() {
return instanceId;
}
+
+ public void setNodeConnectionStatuses(List connectionStatuses) {
+ this.nodeStatuses = connectionStatuses;
+ }
+
+ public List getNodeConnectionStatuses() {
+ return this.nodeStatuses;
+ }
+
+ public List getComponentRevisions() {
+ return componentRevisions;
+ }
+
+ public void setComponentRevisions(List componentRevisions) {
+ this.componentRevisions = componentRevisions;
+ }
}
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 9501b48a19..f1eba521c5 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,6 +16,8 @@
*/
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;
@@ -27,7 +29,7 @@ public class AdaptedHeartbeat {
private NodeIdentifier nodeIdentifier;
private byte[] payload;
- private boolean primary;
+ private Set roles;
private NodeConnectionStatus connectionStatus;
public AdaptedHeartbeat() {
@@ -42,12 +44,12 @@ public class AdaptedHeartbeat {
this.nodeIdentifier = nodeIdentifier;
}
- public boolean isPrimary() {
- return primary;
+ public Set getRoles() {
+ return roles;
}
- public void setPrimary(boolean primary) {
- this.primary = primary;
+ public void setRoles(Set roles) {
+ this.roles = roles;
}
public void setConnectionStatus(NodeConnectionStatus 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 f9ec3b10ea..9cfac2cc7b 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,14 +17,36 @@
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;
public class AdaptedNodeConnectionStatus {
+ private Long updateId;
+ private NodeIdentifier nodeId;
private NodeConnectionState state;
private DisconnectionCode disconnectCode;
private String disconnectReason;
private Long connectionRequestTime;
+ private Set roles;
+
+ public Long getUpdateId() {
+ return updateId;
+ }
+
+ public void setUpdateId(Long updateId) {
+ this.updateId = updateId;
+ }
+
+ public NodeIdentifier getNodeId() {
+ return nodeId;
+ }
+
+ public void setNodeId(NodeIdentifier nodeId) {
+ this.nodeId = nodeId;
+ }
public NodeConnectionState getState() {
return state;
@@ -57,4 +79,12 @@ public class AdaptedNodeConnectionStatus {
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/ConnectionResponseAdapter.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionResponseAdapter.java
index a1bc907c04..ca98a8685a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionResponseAdapter.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionResponseAdapter.java
@@ -34,6 +34,8 @@ public class ConnectionResponseAdapter extends XmlAdapter {
+public class DataFlowAdapter extends XmlAdapter {
@Override
- public AdaptedDataFlow marshal(final StandardDataFlow df) {
-
+ public AdaptedDataFlow marshal(final DataFlow df) {
final AdaptedDataFlow aDf = new AdaptedDataFlow();
if (df != null) {
@@ -39,7 +39,7 @@ public class DataFlowAdapter extends XmlAdapter {
aHb.setPayload(hb.getPayload());
// set leader flag
- aHb.setPrimary(hb.isPrimary());
+ aHb.setRoles(hb.getRoles());
// set connected flag
aHb.setConnectionStatus(hb.getConnectionStatus());
@@ -46,7 +46,7 @@ public class HeartbeatAdapter extends XmlAdapter {
@Override
public Heartbeat unmarshal(final AdaptedHeartbeat aHb) {
- return new Heartbeat(aHb.getNodeIdentifier(), aHb.isPrimary(), aHb.getConnectionStatus(), aHb.getPayload());
+ return new Heartbeat(aHb.getNodeIdentifier(), aHb.getRoles(), 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/JaxbProtocolUtils.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/JaxbProtocolUtils.java
index cbb6747e2a..eb680cb24f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/JaxbProtocolUtils.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/JaxbProtocolUtils.java
@@ -37,5 +37,4 @@ public final class JaxbProtocolUtils {
throw new RuntimeException("Unable to create JAXBContext.", e);
}
}
-
}
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 e2c302dcda..0093c3e741 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
@@ -25,16 +25,25 @@ public class NodeConnectionStatusAdapter extends XmlAdapternull
+ * @return the DN of the Coordinator, if it is available or null
* otherwise
*/
- public String getClusterManagerDN() {
- return clusterManagerDN;
+ public String getCoordinatorDN() {
+ return coordinatorDN;
}
@Override
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/NodeStatusChangeMessage.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/NodeStatusChangeMessage.java
index 7a99d0e600..9e8fae0dff 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/NodeStatusChangeMessage.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/NodeStatusChangeMessage.java
@@ -14,36 +14,34 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.nifi.cluster.protocol.message;
import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
+import org.apache.nifi.cluster.protocol.jaxb.message.NodeConnectionStatusAdapter;
+import org.apache.nifi.cluster.protocol.jaxb.message.NodeIdentifierAdapter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
/**
* Message to indicate that the status of a node in the cluster has changed
*/
@XmlRootElement(name = "nodeStatusChange")
public class NodeStatusChangeMessage extends ProtocolMessage {
+ private static final Logger logger = LoggerFactory.getLogger(NodeStatusChangeMessage.class);
+
private NodeConnectionStatus connectionStatus;
private NodeIdentifier nodeId;
- private Long statusUpdateId = -1L;
@Override
public MessageType getType() {
return MessageType.NODE_STATUS_CHANGE;
}
- public void setNodeConnectionStatus(final NodeConnectionStatus status) {
- this.connectionStatus = status;
- }
-
- public NodeConnectionStatus getNodeConnectionStatus() {
- return connectionStatus;
- }
-
+ @XmlJavaTypeAdapter(NodeIdentifierAdapter.class)
public NodeIdentifier getNodeId() {
return nodeId;
}
@@ -52,11 +50,17 @@ public class NodeStatusChangeMessage extends ProtocolMessage {
this.nodeId = nodeId;
}
- public Long getStatusUpdateIdentifier() {
- return statusUpdateId;
+ @XmlJavaTypeAdapter(NodeConnectionStatusAdapter.class)
+ public NodeConnectionStatus getNodeConnectionStatus() {
+ return connectionStatus;
}
- public void setStatusUpdateIdentifier(Long statusUpdateId) {
- this.statusUpdateId = statusUpdateId;
+ public void setNodeConnectionStatus(final NodeConnectionStatus status) {
+ this.connectionStatus = status;
+ }
+
+ @Override
+ public String toString() {
+ return "NodeStatusChangeMessage[nodeId=" + nodeId + ", status=" + getNodeConnectionStatus() + "]";
}
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionRequestMessage.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionRequestMessage.java
index bd00346392..42084988a8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionRequestMessage.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionRequestMessage.java
@@ -16,9 +16,13 @@
*/
package org.apache.nifi.cluster.protocol.message;
+import java.util.List;
+
import javax.xml.bind.annotation.XmlRootElement;
import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
+import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus;
+import org.apache.nifi.cluster.protocol.ComponentRevision;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.cluster.protocol.StandardDataFlow;
import org.apache.nifi.cluster.protocol.jaxb.message.NodeIdentifierAdapter;
@@ -34,6 +38,8 @@ public class ReconnectionRequestMessage extends ProtocolMessage {
private Integer managerRemoteSiteListeningPort;
private Boolean managerRemoteSiteCommsSecure;
private String instanceId;
+ private List nodeStatuses;
+ private List componentRevisions;
public ReconnectionRequestMessage() {
}
@@ -91,4 +97,20 @@ public class ReconnectionRequestMessage extends ProtocolMessage {
public String getInstanceId() {
return instanceId;
}
+
+ public void setNodeConnectionStatuses(List statuses) {
+ this.nodeStatuses = statuses;
+ }
+
+ public List getNodeConnectionStatuses() {
+ return nodeStatuses;
+ }
+
+ public List getComponentRevisions() {
+ return componentRevisions;
+ }
+
+ public void setComponentRevisions(List componentRevisions) {
+ this.componentRevisions = componentRevisions;
+ }
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/MulticastConfigurationFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/MulticastConfigurationFactoryBean.java
deleted file mode 100644
index 460d3bcfc4..0000000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/MulticastConfigurationFactoryBean.java
+++ /dev/null
@@ -1,60 +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.protocol.spring;
-
-import java.util.concurrent.TimeUnit;
-
-import org.apache.nifi.io.socket.multicast.MulticastConfiguration;
-import org.apache.nifi.util.FormatUtils;
-import org.apache.nifi.util.NiFiProperties;
-import org.springframework.beans.factory.FactoryBean;
-
-/**
- * Factory bean for creating a singleton MulticastConfiguration instance.
- */
-public class MulticastConfigurationFactoryBean implements FactoryBean {
-
- private MulticastConfiguration configuration;
- private NiFiProperties properties;
-
- @Override
- public Object getObject() throws Exception {
- if (configuration == null) {
- configuration = new MulticastConfiguration();
-
- final int timeout = (int) FormatUtils.getTimeDuration(properties.getClusterProtocolSocketTimeout(), TimeUnit.MILLISECONDS);
- configuration.setSocketTimeout(timeout);
- configuration.setReuseAddress(true);
- }
- return configuration;
-
- }
-
- @Override
- public Class getObjectType() {
- return MulticastConfiguration.class;
- }
-
- @Override
- public boolean isSingleton() {
- return true;
- }
-
- public void setProperties(NiFiProperties properties) {
- this.properties = properties;
- }
-}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/ServerSocketConfigurationFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/ServerSocketConfigurationFactoryBean.java
index c41aeff599..ae4e70d509 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/ServerSocketConfigurationFactoryBean.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/ServerSocketConfigurationFactoryBean.java
@@ -16,15 +16,14 @@
*/
package org.apache.nifi.cluster.protocol.spring;
-import java.util.concurrent.TimeUnit;
-
import org.apache.nifi.io.socket.SSLContextFactory;
import org.apache.nifi.io.socket.ServerSocketConfiguration;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.util.NiFiProperties;
-
import org.springframework.beans.factory.FactoryBean;
+import java.util.concurrent.TimeUnit;
+
/**
* Factory bean for creating a singleton ServerSocketConfiguration instance.
*/
@@ -39,7 +38,7 @@ public class ServerSocketConfigurationFactoryBean implements FactoryBean
-
-
-
-
-
-
+
-
+
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
+
+
+
-
-
+
+
-
-
+
+
@@ -92,19 +75,6 @@
-
+
-
-
-
-
-
-
-
-
-
-
-
-
-
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java
deleted file mode 100644
index 82d09596a3..0000000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java
+++ /dev/null
@@ -1,133 +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.protocol.impl;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.cluster.protocol.ProtocolHandler;
-import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext;
-import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils;
-import org.apache.nifi.cluster.protocol.message.FlowRequestMessage;
-import org.apache.nifi.cluster.protocol.message.FlowResponseMessage;
-import org.apache.nifi.cluster.protocol.message.PingMessage;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-import org.apache.nifi.io.socket.ServerSocketConfiguration;
-import org.apache.nifi.io.socket.SocketConfiguration;
-import org.junit.After;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.fail;
-import org.junit.Before;
-import org.junit.Test;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-public class ClusterManagerProtocolSenderImplTest {
-
- private InetAddress address;
-
- private int port;
-
- private SocketProtocolListener listener;
-
- private ClusterManagerProtocolSenderImpl sender;
-
- private ProtocolHandler mockHandler;
-
- @Before
- public void setup() throws IOException {
-
- address = InetAddress.getLocalHost();
- ServerSocketConfiguration serverSocketConfiguration = new ServerSocketConfiguration();
- serverSocketConfiguration.setSocketTimeout(2000);
-
- mockHandler = mock(ProtocolHandler.class);
-
- ProtocolContext protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT);
-
- listener = new SocketProtocolListener(5, 0, serverSocketConfiguration, protocolContext);
- listener.addHandler(mockHandler);
- listener.start();
-
- port = listener.getPort();
-
- SocketConfiguration socketConfiguration = new SocketConfiguration();
- sender = new ClusterManagerProtocolSenderImpl(socketConfiguration, protocolContext);
- }
-
- @After
- public void teardown() throws IOException {
- if (listener.isRunning()) {
- listener.stop();
- }
- }
-
- @Test
- public void testRequestFlow() throws Exception {
-
- when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
- when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(new FlowResponseMessage());
- FlowRequestMessage request = new FlowRequestMessage();
- request.setNodeId(new NodeIdentifier("id", "api-address", 1, "localhost", port, "localhost", 3821, false));
- FlowResponseMessage response = sender.requestFlow(request);
- assertNotNull(response);
- }
-
- @Test
- public void testRequestFlowWithBadResponseMessage() throws Exception {
-
- when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
- when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(new PingMessage());
- FlowRequestMessage request = new FlowRequestMessage();
- request.setNodeId(new NodeIdentifier("id", "api-address", 1, "localhost", port, "localhost", 3821, false));
- try {
- sender.requestFlow(request);
- fail("failed to throw exception");
- } catch (ProtocolException pe) {
- }
-
- }
-
- @Test
- public void testRequestFlowDelayedResponse() throws Exception {
-
- final int time = 250;
- sender.getSocketConfiguration().setSocketTimeout(time);
-
- when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
- when(mockHandler.handle(any(ProtocolMessage.class))).thenAnswer(new Answer() {
- @Override
- public FlowResponseMessage answer(InvocationOnMock invocation) throws Throwable {
- Thread.sleep(time * 3);
- return new FlowResponseMessage();
- }
- });
- FlowRequestMessage request = new FlowRequestMessage();
- request.setNodeId(new NodeIdentifier("id", "api-address", 1, "localhost", port, "localhost", 3821, false));
- try {
- sender.requestFlow(request);
- fail("failed to throw exception");
- } catch (ProtocolException pe) {
- }
- }
-
-}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscoveryTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscoveryTest.java
deleted file mode 100644
index 90817b2b85..0000000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscoveryTest.java
+++ /dev/null
@@ -1,134 +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.protocol.impl;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.net.DatagramPacket;
-import java.net.InetSocketAddress;
-import java.net.MulticastSocket;
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller;
-import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext;
-import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils;
-import org.apache.nifi.cluster.protocol.message.PingMessage;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-import org.apache.nifi.cluster.protocol.message.ServiceBroadcastMessage;
-import org.apache.nifi.io.socket.multicast.MulticastConfiguration;
-import org.apache.nifi.io.socket.multicast.MulticastUtils;
-import org.junit.After;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-
-/**
- */
-public class ClusterServiceDiscoveryTest {
-
- private ClusterServiceDiscovery discovery;
-
- private String serviceName;
-
- private MulticastSocket socket;
-
- private InetSocketAddress multicastAddress;
-
- private MulticastConfiguration configuration;
-
- private ProtocolContext protocolContext;
-
- @Before
- public void setup() throws Exception {
-
- serviceName = "some-service";
- multicastAddress = new InetSocketAddress("225.1.1.1", 22222);
- configuration = new MulticastConfiguration();
-
- protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT);
-
- discovery = new ClusterServiceDiscovery(serviceName, multicastAddress, configuration, protocolContext);
- discovery.start();
-
- socket = MulticastUtils.createMulticastSocket(multicastAddress.getPort(), configuration);
- }
-
- @After
- public void teardown() throws IOException {
- try {
- if (discovery.isRunning()) {
- discovery.stop();
- }
- } finally {
- MulticastUtils.closeQuietly(socket);
- }
- }
-
- @Ignore("Test needs to be fixed. Requires an active network connection")
- @Test
- public void testGetAddressOnStartup() {
- assertNull(discovery.getService());
- }
-
- @Ignore("This test has an NPE after ignoring another...perhaps has a bad inter-test dependency")
- @Test
- public void testGetAddressAfterBroadcast() throws Exception {
-
- ServiceBroadcastMessage msg = new ServiceBroadcastMessage();
- msg.setServiceName("some-service");
- msg.setAddress("3.3.3.3");
- msg.setPort(1234);
-
- // marshal message to output stream
- ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller();
- ByteArrayOutputStream baos = new ByteArrayOutputStream();
- marshaller.marshal(msg, baos);
- byte[] requestPacketBytes = baos.toByteArray();
- DatagramPacket packet = new DatagramPacket(requestPacketBytes, requestPacketBytes.length, multicastAddress);
- socket.send(packet);
-
- Thread.sleep(250);
-
- InetSocketAddress updatedAddress = discovery.getService().getServiceAddress();
- assertEquals("some-service", discovery.getServiceName());
- assertEquals("3.3.3.3", updatedAddress.getHostName());
- assertEquals(1234, updatedAddress.getPort());
-
- }
-
- @Ignore("Test needs to be fixed. Requires an active network connection")
- @Test
- public void testBadBroadcastMessage() throws Exception {
-
- ProtocolMessage msg = new PingMessage();
-
- // marshal message to output stream
- ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller();
- ByteArrayOutputStream baos = new ByteArrayOutputStream();
- marshaller.marshal(msg, baos);
- byte[] requestPacketBytes = baos.toByteArray();
- DatagramPacket packet = new DatagramPacket(requestPacketBytes, requestPacketBytes.length, multicastAddress);
- socket.send(packet);
-
- Thread.sleep(250);
-
- assertNull(discovery.getService());
-
- }
-
-}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java
deleted file mode 100644
index ea4015044d..0000000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java
+++ /dev/null
@@ -1,121 +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.protocol.impl;
-
-import java.net.InetSocketAddress;
-import java.util.concurrent.TimeUnit;
-import org.apache.nifi.io.socket.multicast.DiscoverableService;
-import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import org.junit.Before;
-import org.junit.Test;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-import org.mockito.stubbing.OngoingStubbing;
-
-public class ClusterServiceLocatorTest {
-
- private ClusterServiceDiscovery mockServiceDiscovery;
-
- private int fixedPort;
-
- private DiscoverableService fixedService;
-
- private ClusterServiceLocator serviceDiscoveryLocator;
-
- private ClusterServiceLocator serviceDiscoveryFixedPortLocator;
-
- private ClusterServiceLocator fixedServiceLocator;
-
- @Before
- public void setup() throws Exception {
-
- fixedPort = 1;
- mockServiceDiscovery = mock(ClusterServiceDiscovery.class);
- fixedService = new DiscoverableServiceImpl("some-service", InetSocketAddress.createUnresolved("some-host", 20));
-
- serviceDiscoveryLocator = new ClusterServiceLocator(mockServiceDiscovery);
- serviceDiscoveryFixedPortLocator = new ClusterServiceLocator(mockServiceDiscovery, fixedPort);
- fixedServiceLocator = new ClusterServiceLocator(fixedService);
-
- }
-
- @Test
- public void getServiceWhenServiceDiscoveryNotStarted() {
- assertNull(serviceDiscoveryLocator.getService());
- }
-
- @Test
- public void getServiceWhenServiceDiscoveryFixedPortNotStarted() {
- assertNull(serviceDiscoveryLocator.getService());
- }
-
- @Test
- public void getServiceWhenFixedServiceNotStarted() {
- assertEquals(fixedService, fixedServiceLocator.getService());
- }
-
- @Test
- public void getServiceNotOnFirstAttempt() {
-
- ClusterServiceLocator.AttemptsConfig config = new ClusterServiceLocator.AttemptsConfig();
- config.setNumAttempts(2);
- config.setTimeBetweenAttempsUnit(TimeUnit.SECONDS);
- config.setTimeBetweenAttempts(1);
-
- serviceDiscoveryLocator.setAttemptsConfig(config);
-
- OngoingStubbing stubbing = null;
- for (int i = 0; i < config.getNumAttempts() - 1; i++) {
- if (stubbing == null) {
- stubbing = when(mockServiceDiscovery.getService()).thenReturn(null);
- } else {
- stubbing.thenReturn(null);
- }
- }
- stubbing.thenReturn(fixedService);
-
- assertEquals(fixedService, serviceDiscoveryLocator.getService());
-
- }
-
- @Test
- public void getServiceNotOnFirstAttemptWithFixedPort() {
-
- ClusterServiceLocator.AttemptsConfig config = new ClusterServiceLocator.AttemptsConfig();
- config.setNumAttempts(2);
- config.setTimeBetweenAttempsUnit(TimeUnit.SECONDS);
- config.setTimeBetweenAttempts(1);
-
- serviceDiscoveryFixedPortLocator.setAttemptsConfig(config);
-
- OngoingStubbing stubbing = null;
- for (int i = 0; i < config.getNumAttempts() - 1; i++) {
- if (stubbing == null) {
- stubbing = when(mockServiceDiscovery.getService()).thenReturn(null);
- } else {
- stubbing.thenReturn(null);
- }
- }
- stubbing.thenReturn(fixedService);
-
- InetSocketAddress resultAddress = InetSocketAddress.createUnresolved(fixedService.getServiceAddress().getHostName(), fixedPort);
- DiscoverableService resultService = new DiscoverableServiceImpl(fixedService.getServiceName(), resultAddress);
- assertEquals(resultService, serviceDiscoveryFixedPortLocator.getService());
- }
-}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java
deleted file mode 100644
index 24dd17dee5..0000000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java
+++ /dev/null
@@ -1,131 +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.protocol.impl;
-
-import java.net.InetSocketAddress;
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.cluster.protocol.ProtocolHandler;
-import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext;
-import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-import org.apache.nifi.cluster.protocol.message.ServiceBroadcastMessage;
-import org.apache.nifi.io.socket.multicast.DiscoverableService;
-import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl;
-import org.apache.nifi.io.socket.multicast.MulticastConfiguration;
-import org.junit.After;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-
-/**
- */
-public class ClusterServicesBroadcasterTest {
-
- private ClusterServicesBroadcaster broadcaster;
-
- private MulticastProtocolListener listener;
-
- private DummyProtocolHandler handler;
-
- private InetSocketAddress multicastAddress;
-
- private DiscoverableService broadcastedService;
-
- private ProtocolContext protocolContext;
-
- private MulticastConfiguration configuration;
-
- @Before
- public void setup() throws Exception {
-
- broadcastedService = new DiscoverableServiceImpl("some-service", new InetSocketAddress("localhost", 11111));
-
- multicastAddress = new InetSocketAddress("225.1.1.1", 22222);
-
- configuration = new MulticastConfiguration();
-
- protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT);
-
- broadcaster = new ClusterServicesBroadcaster(multicastAddress, configuration, protocolContext, "500 ms");
- broadcaster.addService(broadcastedService);
-
- handler = new DummyProtocolHandler();
- listener = new MulticastProtocolListener(5, multicastAddress, configuration, protocolContext);
- listener.addHandler(handler);
- }
-
- @After
- public void teardown() {
-
- if (broadcaster.isRunning()) {
- broadcaster.stop();
- }
-
- try {
- if (listener.isRunning()) {
- listener.stop();
- }
- } catch (Exception ex) {
- ex.printStackTrace(System.out);
- }
-
- }
-
- @Ignore("fails needs to be fixed")
- @Test
- public void testBroadcastReceived() throws Exception {
-
- broadcaster.start();
- listener.start();
-
- Thread.sleep(1000);
-
- listener.stop();
-
- assertNotNull(handler.getProtocolMessage());
- assertEquals(ProtocolMessage.MessageType.SERVICE_BROADCAST, handler.getProtocolMessage().getType());
- final ServiceBroadcastMessage msg = (ServiceBroadcastMessage) handler.getProtocolMessage();
- assertEquals(broadcastedService.getServiceName(), msg.getServiceName());
- assertEquals(broadcastedService.getServiceAddress().getHostName(), msg.getAddress());
- assertEquals(broadcastedService.getServiceAddress().getPort(), msg.getPort());
- }
-
- private class DummyProtocolHandler implements ProtocolHandler {
-
- private ProtocolMessage protocolMessage;
-
- @Override
- public boolean canHandle(ProtocolMessage msg) {
- return true;
- }
-
- @Override
- public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException {
- this.protocolMessage = msg;
- return null;
- }
-
- public ProtocolMessage getProtocolMessage() {
- return protocolMessage;
- }
-
- }
-
-}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java
deleted file mode 100644
index 690d416d28..0000000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java
+++ /dev/null
@@ -1,168 +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.protocol.impl;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.net.DatagramPacket;
-import java.net.InetSocketAddress;
-import java.net.MulticastSocket;
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.cluster.protocol.ProtocolHandler;
-import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller;
-import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext;
-import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils;
-import org.apache.nifi.cluster.protocol.message.MulticastProtocolMessage;
-import org.apache.nifi.cluster.protocol.message.PingMessage;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-import org.apache.nifi.io.socket.multicast.MulticastConfiguration;
-import org.apache.nifi.io.socket.multicast.MulticastUtils;
-import org.junit.After;
-import static org.junit.Assert.assertEquals;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-
-public class MulticastProtocolListenerTest {
-
- private MulticastProtocolListener listener;
-
- private MulticastSocket socket;
-
- private InetSocketAddress address;
-
- private MulticastConfiguration configuration;
-
- private ProtocolContext protocolContext;
-
- @Before
- public void setup() throws Exception {
-
- address = new InetSocketAddress("226.1.1.1", 60000);
- configuration = new MulticastConfiguration();
-
- protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT);
-
- listener = new MulticastProtocolListener(5, address, configuration, protocolContext);
- listener.start();
-
- socket = MulticastUtils.createMulticastSocket(address.getPort(), configuration);
- }
-
- @After
- public void teardown() throws IOException {
- try {
- if (listener.isRunning()) {
- listener.stop();
- }
- } finally {
- MulticastUtils.closeQuietly(socket);
- }
- }
-
- @Ignore("Test needs to be reworked. Fails if on a system without actiev network connection")
- @Test
- public void testBadRequest() throws Exception {
- DelayedProtocolHandler handler = new DelayedProtocolHandler(0);
- listener.addHandler(handler);
- DatagramPacket packet = new DatagramPacket(new byte[]{5}, 1, address);
- socket.send(packet);
- Thread.sleep(250);
- assertEquals(0, handler.getMessages().size());
- }
-
- @Ignore("this test works sometimes and fails others - needs work to be reliable")
- @Test
- public void testRequest() throws Exception {
-
- ReflexiveProtocolHandler handler = new ReflexiveProtocolHandler();
- listener.addHandler(handler);
-
- ProtocolMessage msg = new PingMessage();
- MulticastProtocolMessage multicastMsg = new MulticastProtocolMessage("some-id", msg);
-
- // marshal message to output stream
- ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller();
- ByteArrayOutputStream baos = new ByteArrayOutputStream();
- marshaller.marshal(multicastMsg, baos);
- byte[] requestPacketBytes = baos.toByteArray();
- DatagramPacket packet = new DatagramPacket(requestPacketBytes, requestPacketBytes.length, address);
- socket.send(packet);
-
- Thread.sleep(250);
- assertEquals(1, handler.getMessages().size());
- assertEquals(msg.getType(), handler.getMessages().get(0).getType());
-
- }
-
- private class ReflexiveProtocolHandler implements ProtocolHandler {
-
- private List messages = new ArrayList<>();
-
- @Override
- public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException {
- messages.add(msg);
- return msg;
- }
-
- @Override
- public boolean canHandle(ProtocolMessage msg) {
- return true;
- }
-
- public List getMessages() {
- return messages;
- }
-
- }
-
- private class DelayedProtocolHandler implements ProtocolHandler {
-
- private int delay = 0;
-
- private List messages = new ArrayList<>();
-
- public DelayedProtocolHandler(int delay) {
- this.delay = delay;
- }
-
- @Override
- public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException {
- try {
- messages.add(msg);
- Thread.sleep(delay);
- return null;
- } catch (final InterruptedException ie) {
- throw new ProtocolException(ie);
- }
-
- }
-
- @Override
- public boolean canHandle(ProtocolMessage msg) {
- return true;
- }
-
- public List getMessages() {
- return messages;
- }
-
- }
-}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java
deleted file mode 100644
index bd57fe4904..0000000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java
+++ /dev/null
@@ -1,168 +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.protocol.impl;
-
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.UUID;
-
-import org.apache.nifi.cluster.protocol.ConnectionRequest;
-import org.apache.nifi.cluster.protocol.ConnectionResponse;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.cluster.protocol.ProtocolHandler;
-import org.apache.nifi.cluster.protocol.StandardDataFlow;
-import org.apache.nifi.cluster.protocol.UnknownServiceAddressException;
-import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext;
-import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils;
-import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage;
-import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage;
-import org.apache.nifi.cluster.protocol.message.PingMessage;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-import org.apache.nifi.io.socket.ServerSocketConfiguration;
-import org.apache.nifi.io.socket.SocketConfiguration;
-import org.apache.nifi.io.socket.multicast.DiscoverableService;
-import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-@Ignore("Randomly tests... probably timing-specific")
-public class NodeProtocolSenderImplTest {
-
- private SocketProtocolListener listener;
-
- private NodeProtocolSenderImpl sender;
-
- private DiscoverableService service;
-
- private ServerSocketConfiguration serverSocketConfiguration;
-
- private ClusterServiceLocator mockServiceLocator;
-
- private ProtocolHandler mockHandler;
-
- private NodeIdentifier nodeIdentifier;
-
- @Before
- public void setup() throws IOException {
-
- serverSocketConfiguration = new ServerSocketConfiguration();
-
- mockServiceLocator = mock(ClusterServiceLocator.class);
- mockHandler = mock(ProtocolHandler.class);
-
- nodeIdentifier = new NodeIdentifier("1", "localhost", 1234, "localhost", 5678, "localhost", 3821, false);
-
- ProtocolContext protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT);
-
- listener = new SocketProtocolListener(5, 0, serverSocketConfiguration, protocolContext);
- listener.setShutdownListenerSeconds(3);
- listener.addHandler(mockHandler);
- listener.start();
-
- service = new DiscoverableServiceImpl("some-service", new InetSocketAddress("localhost", listener.getPort()));
-
- SocketConfiguration socketConfiguration = new SocketConfiguration();
- socketConfiguration.setReuseAddress(true);
- sender = new NodeProtocolSenderImpl(mockServiceLocator, socketConfiguration, protocolContext);
- }
-
- @After
- public void teardown() throws IOException {
- if (listener.isRunning()) {
- listener.stop();
- }
- }
-
- @Test
- public void testConnect() throws Exception {
-
- when(mockServiceLocator.getService()).thenReturn(service);
- when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
- ConnectionResponseMessage mockMessage = new ConnectionResponseMessage();
- mockMessage.setConnectionResponse(new ConnectionResponse(nodeIdentifier,
- new StandardDataFlow("flow".getBytes("UTF-8"), new byte[0]), null, null, UUID.randomUUID().toString()));
- when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(mockMessage);
-
- ConnectionRequestMessage request = new ConnectionRequestMessage();
- request.setConnectionRequest(new ConnectionRequest(nodeIdentifier));
- ConnectionResponseMessage response = sender.requestConnection(request);
- assertNotNull(response);
- }
-
- @Test(expected = UnknownServiceAddressException.class)
- public void testConnectNoClusterManagerAddress() throws Exception {
-
- when(mockServiceLocator.getService()).thenReturn(null);
- when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
- when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(new ConnectionResponseMessage());
-
- ConnectionRequestMessage request = new ConnectionRequestMessage();
- request.setConnectionRequest(new ConnectionRequest(nodeIdentifier));
-
- sender.requestConnection(request);
- fail("failed to throw exception");
- }
-
- @Test(expected = ProtocolException.class)
- public void testConnectBadResponse() throws Exception {
-
- when(mockServiceLocator.getService()).thenReturn(service);
- when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
- when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(new PingMessage());
-
- ConnectionRequestMessage request = new ConnectionRequestMessage();
- request.setConnectionRequest(new ConnectionRequest(nodeIdentifier));
-
- sender.requestConnection(request);
- fail("failed to throw exception");
-
- }
-
- @Test(expected = ProtocolException.class)
- public void testConnectDelayedResponse() throws Exception {
-
- final int time = 250;
- sender.getSocketConfiguration().setSocketTimeout(time);
- when(mockServiceLocator.getService()).thenReturn(service);
- when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
- when(mockHandler.handle(any(ProtocolMessage.class))).thenAnswer(new Answer() {
- @Override
- public ConnectionResponseMessage answer(InvocationOnMock invocation) throws Throwable {
- Thread.sleep(time * 3);
- return new ConnectionResponseMessage();
- }
- });
- ConnectionRequestMessage request = new ConnectionRequestMessage();
- request.setConnectionRequest(new ConnectionRequest(nodeIdentifier));
-
- sender.requestConnection(request);
- fail("failed to throw exception");
-
- }
-}
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
new file mode 100644
index 0000000000..25ab73a888
--- /dev/null
+++ 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
@@ -0,0 +1,68 @@
+/*
+ * 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.jaxb.message;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.util.Collections;
+import java.util.List;
+
+import javax.xml.bind.JAXBException;
+
+import org.apache.nifi.cluster.coordination.node.DisconnectionCode;
+import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus;
+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.NodeIdentifier;
+import org.apache.nifi.cluster.protocol.StandardDataFlow;
+import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage;
+import org.apache.nifi.web.Revision;
+import org.junit.Test;
+
+public class TestJaxbProtocolUtils {
+
+ @Test
+ public void testRoundTripConnectionResponse() throws JAXBException {
+ final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+
+ final ConnectionResponseMessage msg = new ConnectionResponseMessage();
+ final NodeIdentifier nodeId = new NodeIdentifier("id", "localhost", 8000, "localhost", 8001, "localhost", 8002, true);
+ final DataFlow dataFlow = new StandardDataFlow(new byte[0], new byte[0]);
+ final List nodeStatuses = Collections.singletonList(new NodeConnectionStatus(nodeId, DisconnectionCode.NOT_YET_CONNECTED));
+ final List componentRevisions = Collections.singletonList(ComponentRevision.fromRevision(new Revision(8L, "client-1", "component-1")));
+ msg.setConnectionResponse(new ConnectionResponse(nodeId, dataFlow, 80, false, "instance-1", nodeStatuses, componentRevisions));
+
+ JaxbProtocolUtils.JAXB_CONTEXT.createMarshaller().marshal(msg, baos);
+ final Object unmarshalled = JaxbProtocolUtils.JAXB_CONTEXT.createUnmarshaller().unmarshal(new ByteArrayInputStream(baos.toByteArray()));
+ assertTrue(unmarshalled instanceof ConnectionResponseMessage);
+ final ConnectionResponseMessage unmarshalledMsg = (ConnectionResponseMessage) unmarshalled;
+
+ final List revisions = msg.getConnectionResponse().getComponentRevisions();
+ assertEquals(1, revisions.size());
+ assertEquals(8L, revisions.get(0).getVersion().longValue());
+ assertEquals("client-1", revisions.get(0).getClientId());
+ assertEquals("component-1", revisions.get(0).getComponentId());
+
+ assertEquals(revisions, unmarshalledMsg.getConnectionResponse().getComponentRevisions());
+ }
+
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/client/MulticastTestClient.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/client/MulticastTestClient.java
deleted file mode 100644
index 0b70c61c9a..0000000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/client/MulticastTestClient.java
+++ /dev/null
@@ -1,151 +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.client;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.net.DatagramPacket;
-import java.net.InetSocketAddress;
-import java.net.MulticastSocket;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.Locale;
-import java.util.Timer;
-import java.util.TimerTask;
-
-import org.apache.nifi.cluster.protocol.ProtocolContext;
-import org.apache.nifi.cluster.protocol.ProtocolException;
-import org.apache.nifi.cluster.protocol.ProtocolHandler;
-import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller;
-import org.apache.nifi.cluster.protocol.impl.MulticastProtocolListener;
-import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext;
-import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils;
-import org.apache.nifi.cluster.protocol.message.PingMessage;
-import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
-import org.apache.nifi.io.socket.multicast.MulticastConfiguration;
-import org.apache.nifi.io.socket.multicast.MulticastUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Simple multicast test client that sends ping messages to a group address.
- */
-public class MulticastTestClient {
-
- private static final Logger logger = LoggerFactory.getLogger(MulticastTestClient.class);
-
- private static final int PING_DELAY_SECONDS = 3;
-
- public static void main(final String... args) throws IOException {
-
- String group = System.getProperty("group", "225.0.0.0");
- if (group == null) {
- System.out.println("Host system property 'group' was not given.");
- return;
- }
- group = group.trim();
- if (group.length() == 0) {
- System.out.println("Host system property 'group' must be non-empty.");
- return;
- }
-
- final String portStr = System.getProperty("port", "2222");
- final int port;
- try {
- port = Integer.parseInt(portStr);
- } catch (final NumberFormatException nfe) {
- System.out.println("Port system property 'port' was not a valid port.");
- return;
- }
-
- logger.info(String.format("Pinging every %s seconds using multicast address: %s:%s.", PING_DELAY_SECONDS, group, port));
- logger.info("Override defaults by using system properties '-Dgroup=' and '-Dport='.");
- logger.info("The test client may be stopped by entering a newline at the command line.");
-
- final InetSocketAddress addr = new InetSocketAddress(group, port);
- final ProtocolContext protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT);
- final MulticastConfiguration multicastConfig = new MulticastConfiguration();
- multicastConfig.setReuseAddress(true);
-
- // setup listener
- final MulticastProtocolListener listener = new MulticastProtocolListener(1, addr, multicastConfig, protocolContext);
- listener.addHandler(new ProtocolHandler() {
- @Override
- public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException {
- final PingMessage pingMsg = (PingMessage) msg;
- final SimpleDateFormat sdf = new SimpleDateFormat("HH:mm:ss", Locale.US);
- logger.info("Pinged at: " + sdf.format(pingMsg.getDate()));
- return null;
- }
-
- @Override
- public boolean canHandle(ProtocolMessage msg) {
- return true;
- }
- });
-
- // setup socket
- final MulticastSocket multicastSocket = MulticastUtils.createMulticastSocket(multicastConfig);
-
- // setup broadcaster
- final Timer broadcaster = new Timer("Multicast Test Client", /**
- * is daemon *
- */
- true);
-
- try {
-
- // start listening
- listener.start();
-
- // start broadcasting
- broadcaster.schedule(new TimerTask() {
-
- @Override
- public void run() {
- try {
-
- final PingMessage msg = new PingMessage();
- msg.setDate(new Date());
-
- // marshal message to output stream
- final ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller();
- final ByteArrayOutputStream baos = new ByteArrayOutputStream();
- marshaller.marshal(msg, baos);
- final byte[] packetBytes = baos.toByteArray();
-
- // send message
- final DatagramPacket packet = new DatagramPacket(packetBytes, packetBytes.length, addr);
- multicastSocket.send(packet);
-
- } catch (final Exception ex) {
- logger.warn("Failed to send message due to: " + ex, ex);
- }
- }
- }, 0, PING_DELAY_SECONDS * 1000);
-
- // block until any input is received
- System.in.read();
-
- } finally {
- broadcaster.cancel();
- if (listener.isRunning()) {
- listener.stop();
- }
- }
- }
-}
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 d6838cdc0d..be9559dad9 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
@@ -17,11 +17,6 @@
package org.apache.nifi.cluster.coordination.heartbeat;
-import java.util.Map;
-import java.util.Properties;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-
import org.apache.nifi.cluster.coordination.ClusterCoordinator;
import org.apache.nifi.cluster.coordination.node.DisconnectionCode;
import org.apache.nifi.cluster.coordination.node.NodeConnectionState;
@@ -35,6 +30,11 @@ import org.apache.nifi.util.StopWatch;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
public abstract class AbstractHeartbeatMonitor implements HeartbeatMonitor {
private final int heartbeatIntervalMillis;
@@ -55,9 +55,19 @@ public abstract class AbstractHeartbeatMonitor implements HeartbeatMonitor {
}
@Override
- public final void start() {
+ public synchronized final void start() {
+ if (!stopped) {
+ throw new IllegalStateException("Heartbeat Monitor cannot be started because it is already started");
+ }
+
stopped = false;
- onStart();
+ logger.info("Heartbeat Monitor started");
+
+ try {
+ onStart();
+ } catch (final Exception e) {
+ logger.error("Failed to start Heartbeat Monitor", e);
+ }
this.future = flowEngine.scheduleWithFixedDelay(new Runnable() {
@Override
@@ -73,8 +83,13 @@ public abstract class AbstractHeartbeatMonitor implements HeartbeatMonitor {
}
@Override
- public final void stop() {
+ public synchronized final void stop() {
+ if (stopped) {
+ return;
+ }
+
this.stopped = true;
+ logger.info("Heartbeat Monitor stopped");
try {
if (future != null) {
@@ -112,7 +127,7 @@ public abstract class AbstractHeartbeatMonitor implements HeartbeatMonitor {
final Map latestHeartbeats = getLatestHeartbeats();
if (latestHeartbeats == null || latestHeartbeats.isEmpty()) {
// failed to fetch heartbeats; don't change anything.
- clusterCoordinator.reportEvent(null, Severity.WARNING, "Failed to retrieve any new heartbeat information for nodes. "
+ clusterCoordinator.reportEvent(null, Severity.INFO, "Failed to retrieve any new heartbeat information for nodes. "
+ "Will not make any decisions based on heartbeats.");
return;
}
@@ -133,8 +148,8 @@ public abstract class AbstractHeartbeatMonitor implements HeartbeatMonitor {
logger.info("Finished processing {} heartbeats in {}", latestHeartbeats.size(), procStopWatch.getDuration());
// Disconnect any node that hasn't sent a heartbeat in a long time (8 times the heartbeat interval)
- final long maxMillis = heartbeatIntervalMillis * 1000L * 8;
- final long threshold = latestHeartbeatTime - maxMillis;
+ final long maxMillis = heartbeatIntervalMillis * 8;
+ final long threshold = System.currentTimeMillis() - maxMillis;
for (final NodeHeartbeat heartbeat : latestHeartbeats.values()) {
if (heartbeat.getTimestamp() < threshold) {
clusterCoordinator.requestNodeDisconnect(heartbeat.getNodeIdentifier(), DisconnectionCode.LACK_OF_HEARTBEAT,
@@ -165,52 +180,21 @@ public abstract class AbstractHeartbeatMonitor implements HeartbeatMonitor {
final NodeConnectionStatus connectionStatus = clusterCoordinator.getConnectionStatus(nodeId);
if (connectionStatus == null) {
- final NodeConnectionState hbConnectionState = heartbeat.getConnectionStatus().getState();
- if (hbConnectionState == NodeConnectionState.DISCONNECTED || hbConnectionState == NodeConnectionState.DISCONNECTING) {
- // Node is not part of the cluster. Remove heartbeat and move on.
- removeHeartbeat(nodeId);
- return;
- }
-
// Unknown node. Issue reconnect request
clusterCoordinator.reportEvent(nodeId, Severity.INFO, "Received heartbeat from unknown node. Removing heartbeat and requesting that node connect to cluster.");
removeHeartbeat(nodeId);
- clusterCoordinator.requestNodeConnect(nodeId);
+ clusterCoordinator.requestNodeConnect(nodeId, null);
return;
}
- final DisconnectionCode reportedDisconnectCode = heartbeat.getConnectionStatus().getDisconnectCode();
- if (reportedDisconnectCode != null) {
- // Check if the node is notifying us that it wants to disconnect from the cluster
- final boolean requestingDisconnect;
- switch (reportedDisconnectCode) {
- case MISMATCHED_FLOWS:
- case NODE_SHUTDOWN:
- case STARTUP_FAILURE:
- final NodeConnectionState expectedState = connectionStatus.getState();
- requestingDisconnect = expectedState == NodeConnectionState.CONNECTED || expectedState == NodeConnectionState.CONNECTING;
- break;
- default:
- requestingDisconnect = false;
- break;
- }
-
- if (requestingDisconnect) {
- clusterCoordinator.disconnectionRequestedByNode(nodeId, heartbeat.getConnectionStatus().getDisconnectCode(),
- heartbeat.getConnectionStatus().getDisconnectReason());
- removeHeartbeat(nodeId);
- return;
- }
- }
-
final NodeConnectionState connectionState = connectionStatus.getState();
if (heartbeat.getConnectionStatus().getState() != NodeConnectionState.CONNECTED && connectionState == NodeConnectionState.CONNECTED) {
// Cluster Coordinator believes that node is connected, but node does not believe so.
clusterCoordinator.reportEvent(nodeId, Severity.WARNING, "Received heartbeat from node that thinks it is not yet part of the cluster,"
+ "though the Cluster Coordinator thought it was (node claimed state was " + heartbeat.getConnectionStatus().getState()
+ "). Marking as Disconnected and requesting that Node reconnect to cluster");
- clusterCoordinator.requestNodeConnect(nodeId);
+ clusterCoordinator.requestNodeConnect(nodeId, null);
return;
}
@@ -220,14 +204,14 @@ public abstract class AbstractHeartbeatMonitor implements HeartbeatMonitor {
// we cannot manually reconnect it.
final DisconnectionCode disconnectionCode = connectionStatus.getDisconnectCode();
- if (disconnectionCode == DisconnectionCode.LACK_OF_HEARTBEAT) {
- // record event
+ // Determine whether or not the node should be allowed to be in the cluster still, depending on its reason for disconnection.
+ if (disconnectionCode == DisconnectionCode.LACK_OF_HEARTBEAT || disconnectionCode == DisconnectionCode.UNABLE_TO_COMMUNICATE) {
clusterCoordinator.reportEvent(nodeId, Severity.INFO, "Received heartbeat from node previously "
- + "disconnected due to lack of heartbeat. Issuing reconnection request.");
+ + "disconnected due to " + disconnectionCode + ". Issuing reconnection request.");
- clusterCoordinator.requestNodeConnect(nodeId);
+ clusterCoordinator.requestNodeConnect(nodeId, null);
} else {
- // disconnected nodes should not heartbeat, so we need to issue a disconnection request
+ // disconnected nodes should not heartbeat, so we need to issue a disconnection request.
logger.info("Ignoring received heartbeat from disconnected node " + nodeId + ". Issuing disconnection request.");
clusterCoordinator.requestNodeDisconnect(nodeId, connectionStatus.getDisconnectCode(), connectionStatus.getDisconnectReason());
removeHeartbeat(nodeId);
@@ -256,9 +240,7 @@ public abstract class AbstractHeartbeatMonitor implements HeartbeatMonitor {
clusterCoordinator.reportEvent(nodeId, Severity.INFO, "Received first heartbeat from connecting node. Node connected.");
}
- if (heartbeat.isPrimary()) {
- clusterCoordinator.setPrimaryNode(nodeId);
- }
+ 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 f172915c4c..934c81bab4 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
@@ -43,6 +43,7 @@ import org.apache.nifi.cluster.protocol.Heartbeat;
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.ProtocolMessage;
import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType;
@@ -55,7 +56,8 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
- * Uses Apache Curator to monitor heartbeats from nodes
+ * 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 {
protected static final Logger logger = LoggerFactory.getLogger(ClusterProtocolHeartbeatMonitor.class);
@@ -64,7 +66,6 @@ public class ClusterProtocolHeartbeatMonitor extends AbstractHeartbeatMonitor im
private final ZooKeeperClientConfig zkClientConfig;
private final String clusterNodesPath;
- private volatile CuratorFramework curatorClient;
private volatile Map clusterNodeIds = new HashMap<>();
private final String heartbeatAddress;
@@ -82,32 +83,33 @@ public class ClusterProtocolHeartbeatMonitor extends AbstractHeartbeatMonitor im
}
- public ClusterProtocolHeartbeatMonitor(final ClusterCoordinator clusterCoordinator, final Properties properties) {
+ public ClusterProtocolHeartbeatMonitor(final ClusterCoordinator clusterCoordinator, final ProtocolListener protocolListener, final Properties properties) {
super(clusterCoordinator, properties);
+ protocolListener.addHandler(this);
this.zkClientConfig = ZooKeeperClientConfig.createConfig(properties);
this.clusterNodesPath = zkClientConfig.resolvePath("cluster/nodes");
- String hostname = properties.getProperty(NiFiProperties.CLUSTER_MANAGER_ADDRESS);
+ String hostname = properties.getProperty(NiFiProperties.CLUSTER_NODE_ADDRESS);
if (hostname == null) {
try {
hostname = InetAddress.getLocalHost().getHostName();
} catch (UnknownHostException e) {
- throw new RuntimeException("Unable to determine local hostname and the '" + NiFiProperties.CLUSTER_MANAGER_ADDRESS + "' property is not set");
+ throw new RuntimeException("Unable to determine local hostname and the '" + NiFiProperties.CLUSTER_NODE_ADDRESS + "' property is not set");
}
}
- final String port = properties.getProperty(NiFiProperties.CLUSTER_MANAGER_PROTOCOL_PORT);
+ final String port = properties.getProperty(NiFiProperties.CLUSTER_NODE_PROTOCOL_PORT);
if (port == null) {
- throw new RuntimeException("Unable to determine which port Cluster Manager Protocol is listening on because the '"
- + NiFiProperties.CLUSTER_MANAGER_PROTOCOL_PORT + "' property is not set");
+ throw new RuntimeException("Unable to determine which port Cluster Coordinator Protocol is listening on because the '"
+ + NiFiProperties.CLUSTER_NODE_PROTOCOL_PORT + "' property is not set");
}
try {
Integer.parseInt(port);
} catch (final NumberFormatException nfe) {
- throw new RuntimeException("Unable to determine which port Cluster Manager Protocol is listening on because the '"
- + NiFiProperties.CLUSTER_MANAGER_PROTOCOL_PORT + "' property is set to '" + port + "', which is not a valid port number.");
+ throw new RuntimeException("Unable to determine which port Cluster Coordinator Protocol is listening on because the '"
+ + NiFiProperties.CLUSTER_NODE_PROTOCOL_PORT + "' property is set to '" + port + "', which is not a valid port number.");
}
heartbeatAddress = hostname + ":" + port;
@@ -116,10 +118,22 @@ public class ClusterProtocolHeartbeatMonitor extends AbstractHeartbeatMonitor im
@Override
public void onStart() {
final RetryPolicy retryPolicy = new RetryForever(5000);
- curatorClient = CuratorFrameworkFactory.newClient(zkClientConfig.getConnectString(),
+ final CuratorFramework curatorClient = CuratorFrameworkFactory.newClient(zkClientConfig.getConnectString(),
zkClientConfig.getSessionTimeoutMillis(), zkClientConfig.getConnectionTimeoutMillis(), retryPolicy);
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
+ // that if we go the required amount of time without receiving a heartbeat, we do know to mark the node
+ // as disconnected.
+ 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());
+ heartbeatMessages.put(nodeId, heartbeat);
+ }
+
final Thread publishAddress = new Thread(new Runnable() {
@Override
public void run() {
@@ -128,6 +142,8 @@ public class ClusterProtocolHeartbeatMonitor extends AbstractHeartbeatMonitor im
try {
try {
curatorClient.setData().forPath(path, heartbeatAddress.getBytes(StandardCharsets.UTF_8));
+ curatorClient.close();
+ 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
@@ -162,16 +178,8 @@ public class ClusterProtocolHeartbeatMonitor extends AbstractHeartbeatMonitor im
publishAddress.start();
}
- private CuratorFramework getClient() {
- return curatorClient;
- }
-
@Override
public void onStop() {
- final CuratorFramework client = getClient();
- if (client != null) {
- client.close();
- }
}
@Override
@@ -201,7 +209,7 @@ public class ClusterProtocolHeartbeatMonitor extends AbstractHeartbeatMonitor im
final NodeIdentifier nodeId = heartbeat.getNodeIdentifier();
final NodeConnectionStatus connectionStatus = heartbeat.getConnectionStatus();
- final boolean primary = heartbeat.isPrimary();
+ final Set roles = heartbeat.getRoles();
final byte[] payloadBytes = heartbeat.getPayload();
final HeartbeatPayload payload = HeartbeatPayload.unmarshal(payloadBytes);
final int activeThreadCount = payload.getActiveThreadCount();
@@ -210,7 +218,7 @@ public class ClusterProtocolHeartbeatMonitor extends AbstractHeartbeatMonitor im
final long systemStartTime = payload.getSystemStartTime();
final NodeHeartbeat nodeHeartbeat = new StandardNodeHeartbeat(nodeId, System.currentTimeMillis(),
- connectionStatus, primary, flowFileCount, flowFileBytes, activeThreadCount, systemStartTime);
+ connectionStatus, roles, flowFileCount, flowFileBytes, activeThreadCount, systemStartTime);
heartbeatMessages.put(heartbeat.getNodeIdentifier(), nodeHeartbeat);
logger.debug("Received new heartbeat from {}", nodeId);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/CuratorHeartbeatMonitor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/CuratorHeartbeatMonitor.java
deleted file mode 100644
index 8114813171..0000000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/CuratorHeartbeatMonitor.java
+++ /dev/null
@@ -1,376 +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 java.io.ByteArrayInputStream;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-
-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.retry.RetryForever;
-import org.apache.nifi.cluster.coordination.ClusterCoordinator;
-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.apache.nifi.cluster.protocol.message.HeartbeatMessage;
-import org.apache.nifi.controller.cluster.ZooKeeperClientConfig;
-import org.apache.nifi.engine.FlowEngine;
-import org.apache.nifi.reporting.Severity;
-import org.apache.nifi.util.FormatUtils;
-import org.apache.nifi.util.NiFiProperties;
-import org.apache.nifi.util.StopWatch;
-import org.apache.zookeeper.KeeperException.NoNodeException;
-import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Uses Apache Curator to monitor heartbeats from nodes
- */
-public class CuratorHeartbeatMonitor implements HeartbeatMonitor {
- private static final Logger logger = LoggerFactory.getLogger(CuratorHeartbeatMonitor.class);
- private static final Unmarshaller unmarshaller;
-
- private final ClusterCoordinator clusterCoordinator;
- private final ZooKeeperClientConfig zkClientConfig;
- private final String heartbeatPath;
- private final int heartbeatIntervalMillis;
-
- private volatile CuratorFramework curatorClient;
- private volatile ScheduledFuture> future;
- private volatile Map latestHeartbeatMessages;
- private volatile long latestHeartbeatTime;
-
- private final FlowEngine flowEngine = new FlowEngine(1, "Heartbeat Monitor", true);
-
- static {
- try {
- final JAXBContext jaxbContext = JAXBContext.newInstance(HeartbeatMessage.class);
- unmarshaller = jaxbContext.createUnmarshaller();
- } catch (final Exception e) {
- throw new RuntimeException("Failed to create an Unmarshaller for unmarshalling Heartbeat Messages", e);
- }
- }
-
- public CuratorHeartbeatMonitor(final ClusterCoordinator clusterCoordinator, final Properties properties) {
- this.clusterCoordinator = clusterCoordinator;
- this.zkClientConfig = ZooKeeperClientConfig.createConfig(properties);
- this.heartbeatPath = zkClientConfig.resolvePath("cluster/heartbeats");
-
- final String heartbeatInterval = properties.getProperty(NiFiProperties.CLUSTER_PROTOCOL_HEARTBEAT_INTERVAL,
- NiFiProperties.DEFAULT_CLUSTER_PROTOCOL_HEARTBEAT_INTERVAL);
-
- this.heartbeatIntervalMillis = (int) FormatUtils.getTimeDuration(heartbeatInterval, TimeUnit.MILLISECONDS);
- }
-
- @Override
- public void start() {
- final RetryPolicy retryPolicy = new RetryForever(5000);
- curatorClient = CuratorFrameworkFactory.newClient(zkClientConfig.getConnectString(),
- zkClientConfig.getSessionTimeoutMillis(), zkClientConfig.getConnectionTimeoutMillis(), retryPolicy);
- curatorClient.start();
-
- this.future = flowEngine.scheduleWithFixedDelay(new Runnable() {
- @Override
- public void run() {
- try {
- monitorHeartbeats();
- } catch (final Exception e) {
- clusterCoordinator.reportEvent(null, Severity.ERROR, "Failed to process heartbeats from nodes due to " + e.toString());
- logger.error("Failed to process heartbeats", e);
- }
- }
- }, heartbeatIntervalMillis, heartbeatIntervalMillis, TimeUnit.MILLISECONDS);
- }
-
- private CuratorFramework getClient() {
- return curatorClient;
- }
-
- @Override
- public void stop() {
- final CuratorFramework client = getClient();
- if (client != null) {
- client.close();
- }
-
- if (future != null) {
- future.cancel(true);
- }
- }
-
- @Override
- public NodeHeartbeat getLatestHeartbeat(final NodeIdentifier nodeId) {
- return latestHeartbeatMessages.get(nodeId);
- }
-
-
- /**
- * Fetches all of the latest heartbeats from ZooKeeper
- * and updates the Cluster Coordinator as appropriate,
- * based on the heartbeats received.
- *
- * Visible for testing.
- */
- synchronized void monitorHeartbeats() {
- final StopWatch fetchStopWatch = new StopWatch(true);
- final Map latestHeartbeats = fetchHeartbeats();
- if (latestHeartbeats == null || latestHeartbeats.isEmpty()) {
- // failed to fetch heartbeats from ZooKeeper; don't change anything.
- clusterCoordinator.reportEvent(null, Severity.WARNING, "Failed to retrieve any new heartbeat information for nodes from ZooKeeper. "
- + "Will not make any decisions based on heartbeats.");
- return;
- }
-
- this.latestHeartbeatMessages = new HashMap<>(latestHeartbeats);
- fetchStopWatch.stop();
-
- final StopWatch procStopWatch = new StopWatch(true);
- for (final NodeHeartbeat heartbeat : latestHeartbeats.values()) {
- try {
- processHeartbeat(heartbeat);
- } catch (final Exception e) {
- clusterCoordinator.reportEvent(null, Severity.ERROR,
- "Received heartbeat from " + heartbeat.getNodeIdentifier() + " but failed to process heartbeat due to " + e);
- logger.error("Failed to process heartbeat from {} due to {}", heartbeat.getNodeIdentifier(), e.toString());
- logger.error("", e);
- }
- }
-
- procStopWatch.stop();
- logger.info("Finished processing {} heartbeats in {} (fetch took an additional {})",
- latestHeartbeats.size(), procStopWatch.getDuration(), fetchStopWatch.getDuration());
-
- // Disconnect any node that hasn't sent a heartbeat in a long time (8 times the heartbeat interval)
- for (final NodeHeartbeat heartbeat : latestHeartbeats.values()) {
- final long maxMillis = heartbeatIntervalMillis * 1000L * 8;
- final long threshold = latestHeartbeatTime - maxMillis;
-
- if (heartbeat.getTimestamp() < threshold) {
- final int differenceSeconds = (int) TimeUnit.MILLISECONDS.toSeconds(latestHeartbeatTime - heartbeat.getTimestamp());
-
- clusterCoordinator.requestNodeDisconnect(heartbeat.getNodeIdentifier(), DisconnectionCode.LACK_OF_HEARTBEAT,
- "Node has not sent a heartbeat to ZooKeeper in " + differenceSeconds + " seconds");
-
- try {
- removeHeartbeat(heartbeat.getNodeIdentifier());
- } catch (final Exception e) {
- logger.warn("Failed to remove heartbeat for {} due to {}", heartbeat.getNodeIdentifier(), e.toString());
- logger.warn("", e);
- }
- }
- }
- }
-
-
- private void processHeartbeat(final NodeHeartbeat heartbeat) {
- final NodeIdentifier nodeId = heartbeat.getNodeIdentifier();
-
- // Do not process heartbeat if it's blocked by firewall.
- if (clusterCoordinator.isBlockedByFirewall(nodeId.getSocketAddress())) {
- clusterCoordinator.reportEvent(nodeId, Severity.WARNING, "Firewall blocked received heartbeat. Issuing disconnection request.");
-
- // request node to disconnect
- clusterCoordinator.requestNodeDisconnect(nodeId, DisconnectionCode.BLOCKED_BY_FIREWALL, "Blocked by Firewall");
- removeHeartbeat(nodeId);
- return;
- }
-
- final NodeConnectionStatus connectionStatus = clusterCoordinator.getConnectionStatus(nodeId);
- if (connectionStatus == null) {
- final NodeConnectionState hbConnectionState = heartbeat.getConnectionStatus().getState();
- if (hbConnectionState == NodeConnectionState.DISCONNECTED || hbConnectionState == NodeConnectionState.DISCONNECTING) {
- // Node is not part of the cluster. Remove heartbeat and move on.
- removeHeartbeat(nodeId);
- return;
- }
-
- // Unknown node. Issue reconnect request
- clusterCoordinator.reportEvent(nodeId, Severity.INFO, "Received heartbeat from unknown node. Removing heartbeat and requesting that node connect to cluster.");
- removeHeartbeat(nodeId);
-
- clusterCoordinator.requestNodeConnect(nodeId);
- return;
- }
-
- final DisconnectionCode reportedDisconnectCode = heartbeat.getConnectionStatus().getDisconnectCode();
- if (reportedDisconnectCode != null) {
- // Check if the node is notifying us that it wants to disconnect from the cluster
- final boolean requestingDisconnect;
- switch (reportedDisconnectCode) {
- case MISMATCHED_FLOWS:
- case NODE_SHUTDOWN:
- case STARTUP_FAILURE:
- final NodeConnectionState expectedState = connectionStatus.getState();
- requestingDisconnect = expectedState == NodeConnectionState.CONNECTED || expectedState == NodeConnectionState.CONNECTING;
- break;
- default:
- requestingDisconnect = false;
- break;
- }
-
- if (requestingDisconnect) {
- clusterCoordinator.disconnectionRequestedByNode(nodeId, heartbeat.getConnectionStatus().getDisconnectCode(),
- heartbeat.getConnectionStatus().getDisconnectReason());
- removeHeartbeat(nodeId);
- return;
- }
- }
-
- final NodeConnectionState connectionState = connectionStatus.getState();
- if (heartbeat.getConnectionStatus().getState() != NodeConnectionState.CONNECTED && connectionState == NodeConnectionState.CONNECTED) {
- // Cluster Coordinator believes that node is connected, but node does not believe so.
- clusterCoordinator.reportEvent(nodeId, Severity.WARNING, "Received heartbeat from node that thinks it is not yet part of the cluster,"
- + "though the Cluster Coordinator thought it was (node claimed state was " + heartbeat.getConnectionStatus().getState()
- + "). Marking as Disconnected and requesting that Node reconnect to cluster");
- clusterCoordinator.requestNodeConnect(nodeId);
- return;
- }
-
- if (NodeConnectionState.DISCONNECTED == connectionState) {
- // ignore heartbeats from nodes disconnected by means other than lack of heartbeat, unless it is
- // the only node. We allow it if it is the only node because if we have a one-node cluster, then
- // we cannot manually reconnect it.
- final DisconnectionCode disconnectionCode = connectionStatus.getDisconnectCode();
-
- if (disconnectionCode == DisconnectionCode.LACK_OF_HEARTBEAT) {
- // record event
- clusterCoordinator.reportEvent(nodeId, Severity.INFO, "Received heartbeat from node previously "
- + "disconnected due to lack of heartbeat. Issuing reconnection request.");
-
- clusterCoordinator.requestNodeConnect(nodeId);
- } else {
- // disconnected nodes should not heartbeat, so we need to issue a disconnection request
- logger.info("Ignoring received heartbeat from disconnected node " + nodeId + ". Issuing disconnection request.");
- clusterCoordinator.requestNodeDisconnect(nodeId, connectionStatus.getDisconnectCode(), connectionStatus.getDisconnectReason());
- removeHeartbeat(nodeId);
- }
-
- return;
- }
-
- if (NodeConnectionState.DISCONNECTING == connectionStatus.getState()) {
- // ignore spurious heartbeat
- removeHeartbeat(nodeId);
- return;
- }
-
- // first heartbeat causes status change from connecting to connected
- if (NodeConnectionState.CONNECTING == connectionState) {
- final Long connectionRequestTime = connectionStatus.getConnectionRequestTime();
- if (connectionRequestTime != null && heartbeat.getTimestamp() < connectionRequestTime) {
- clusterCoordinator.reportEvent(nodeId, Severity.INFO, "Received heartbeat but ignoring because it was reported before the node was last asked to reconnect.");
- removeHeartbeat(nodeId);
- return;
- }
-
- // connection complete
- clusterCoordinator.finishNodeConnection(nodeId);
- clusterCoordinator.reportEvent(nodeId, Severity.INFO, "Received first heartbeat from connecting node. Node connected.");
- }
-
- if (heartbeat.isPrimary()) {
- clusterCoordinator.setPrimaryNode(nodeId);
- }
- }
-
-
- /**
- * Fetches the latest heartbeats for each node from ZooKeeper.
- * Visible for testing
- */
- Map fetchHeartbeats() {
- logger.debug("Fetching heartbeats from ZooKeeper");
- final List nodeIds;
- try {
- nodeIds = curatorClient.getChildren().forPath(heartbeatPath);
- } catch (final NoNodeException nne) {
- logger.info("Could not find any heartbeats in ZooKeeper because the ZNode " + heartbeatPath + " does not exist");
- return null;
- } catch (final Exception e) {
- logger.error("Failed to obtain heartbeats from ZooKeeper due to {}", e);
- logger.error("", e);
- clusterCoordinator.reportEvent(null, Severity.WARNING, "Failed to obtain heartbeats from ZooKeeper due to " + e);
-
- return null;
- }
-
- logger.debug("Found {} nodes that have emitted heartbeats to ZooKeeper", nodeIds.size());
- final Map latestHeartbeats = new HashMap<>(nodeIds.size());
- for (final String nodeId : nodeIds) {
- final HeartbeatMessage heartbeatMsg;
-
- final String nodeHeartbeatPath = heartbeatPath + "/" + nodeId;
- final Stat stat = new Stat();
- try {
- final byte[] serializedHeartbeatMsg = getClient().getData().storingStatIn(stat).forPath(nodeHeartbeatPath);
- heartbeatMsg = (HeartbeatMessage) unmarshaller.unmarshal(new ByteArrayInputStream(serializedHeartbeatMsg));
- } catch (final Exception e) {
- logger.error("Failed to obtain heartbeat from ZooKeeper for Node with ID {} due to {}", nodeId, e);
- logger.error("", e);
- clusterCoordinator.reportEvent(null, Severity.WARNING, "Failed to obtain heartbeat for Node with ID " + nodeId + " from ZooKeeper due to " + e);
-
- continue;
- }
-
- // update timestamp to be the timestamp that ZooKeeper reports
- final long lastModifiedTime = stat.getMtime();
- if (lastModifiedTime > latestHeartbeatTime) {
- latestHeartbeatTime = lastModifiedTime;
- }
-
- latestHeartbeats.put(heartbeatMsg.getHeartbeat().getNodeIdentifier(), StandardNodeHeartbeat.fromHeartbeatMessage(heartbeatMsg, lastModifiedTime));
- logger.debug("Received heartbeat from Node {}", nodeId);
- }
-
- logger.debug("Fetched {} heartbeats from ZooKeeper", latestHeartbeats.size());
- return latestHeartbeats;
- }
-
- @Override
- public synchronized void removeHeartbeat(final NodeIdentifier nodeId) {
- logger.debug("Deleting heartbeat for node {}", nodeId);
- final String nodeHeartbeatPath = heartbeatPath + "/" + nodeId.getId();
-
- latestHeartbeatMessages.remove(nodeId);
-
- try {
- getClient().delete().forPath(nodeHeartbeatPath);
- logger.info("Removed heartbeat from ZooKeeper for Node {}", nodeId);
- } catch (final NoNodeException e) {
- // node did not exist. Just return.
- logger.debug("Attempted to remove heartbeat for Node with ID {} but no ZNode existed at {}", nodeId, nodeHeartbeatPath);
- return;
- } catch (final Exception e) {
- logger.warn("Failed to remove heartbeat from ZooKeeper for Node {} due to {}", nodeId, e);
- logger.warn("", e);
-
- clusterCoordinator.reportEvent(nodeId, Severity.WARNING, "Failed to remove node's heartbeat from ZooKeeper due to " + e);
- }
- }
-}
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 133bab0876..e455a76db9 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,6 +17,10 @@
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;
@@ -28,18 +32,18 @@ public class StandardNodeHeartbeat implements NodeHeartbeat {
private final NodeIdentifier nodeId;
private final long timestamp;
private final NodeConnectionStatus connectionStatus;
- private final boolean primary;
+ 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 boolean primary, final int flowFileCount, final long flowFileBytes, final int activeThreadCount, final long systemStartTime) {
+ final Set roles, final int flowFileCount, final long flowFileBytes, final int activeThreadCount, final long systemStartTime) {
this.timestamp = timestamp;
this.nodeId = nodeId;
this.connectionStatus = connectionStatus;
- this.primary = primary;
+ this.roles = roles == null ? Collections.emptySet() : Collections.unmodifiableSet(new HashSet<>(roles));
this.flowFileCount = flowFileCount;
this.flowFileBytes = flowFileBytes;
this.activeThreadCount = activeThreadCount;
@@ -62,8 +66,8 @@ public class StandardNodeHeartbeat implements NodeHeartbeat {
}
@Override
- public boolean isPrimary() {
- return primary;
+ public Set getRoles() {
+ return roles;
}
@Override
@@ -92,7 +96,7 @@ public class StandardNodeHeartbeat implements NodeHeartbeat {
final HeartbeatPayload payload = HeartbeatPayload.unmarshal(heartbeat.getPayload());
return new StandardNodeHeartbeat(heartbeat.getNodeIdentifier(), timestamp, heartbeat.getConnectionStatus(),
- heartbeat.isPrimary(), (int) payload.getTotalFlowFileCount(), payload.getTotalFlowFileBytes(),
+ heartbeat.getRoles(), (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/http/StandardHttpResponseMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMerger.java
index 7dc598ea9b..ce2e32a1dd 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMerger.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMerger.java
@@ -35,6 +35,7 @@ import org.apache.nifi.cluster.coordination.http.endpoints.ControllerServicesEnd
import org.apache.nifi.cluster.coordination.http.endpoints.ControllerStatusEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.CountersEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.DropRequestEndpiontMerger;
+import org.apache.nifi.cluster.coordination.http.endpoints.FlowMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.FlowSnippetEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.GroupStatusEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.ListFlowFilesEndpointMerger;
@@ -52,25 +53,15 @@ import org.apache.nifi.cluster.coordination.http.endpoints.ReportingTaskEndpoint
import org.apache.nifi.cluster.coordination.http.endpoints.ReportingTasksEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.StatusHistoryEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.SystemDiagnosticsEndpointMerger;
+import org.apache.nifi.cluster.coordination.http.replication.RequestReplicator;
import org.apache.nifi.cluster.manager.NodeResponse;
-import org.apache.nifi.cluster.manager.StatusMerger;
-import org.apache.nifi.cluster.manager.impl.WebClusterManager;
-import org.apache.nifi.cluster.node.Node.Status;
-import org.apache.nifi.reporting.Bulletin;
-import org.apache.nifi.reporting.BulletinQuery;
-import org.apache.nifi.reporting.ComponentType;
import org.apache.nifi.stream.io.NullOutputStream;
-import org.apache.nifi.web.api.dto.status.ControllerStatusDTO;
-import org.apache.nifi.web.api.entity.ControllerStatusEntity;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class StandardHttpResponseMerger implements HttpResponseMerger {
private Logger logger = LoggerFactory.getLogger(StandardHttpResponseMerger.class);
- private static final int NODE_CONTINUE_STATUS_CODE = 150;
- private final WebClusterManager clusterManager;
-
private static final List endpointMergers = new ArrayList<>();
static {
endpointMergers.add(new ControllerStatusEndpointMerger());
@@ -99,14 +90,10 @@ public class StandardHttpResponseMerger implements HttpResponseMerger {
endpointMergers.add(new StatusHistoryEndpointMerger());
endpointMergers.add(new SystemDiagnosticsEndpointMerger());
endpointMergers.add(new CountersEndpointMerger());
+ endpointMergers.add(new FlowMerger());
}
public StandardHttpResponseMerger() {
- this(null);
- }
-
- public StandardHttpResponseMerger(final WebClusterManager clusterManager) {
- this.clusterManager = clusterManager;
}
@Override
@@ -140,58 +127,9 @@ public class StandardHttpResponseMerger implements HttpResponseMerger {
}
final NodeResponse response = merger.merge(uri, httpMethod, successResponses, problematicResponses, clientResponse);
- if (clusterManager != null) {
- mergeNCMBulletins(response, uri, httpMethod);
- }
-
return response;
}
- /**
- * This method merges bulletins from the NCM. Eventually, the NCM will go away entirely, and
- * at that point, we will completely remove this and the WebClusterManager as a member variable.
- * However, until then, the bulletins from the NCM are important to include, since there is no other
- * node that can include them.
- *
- * @param clientResponse the Node Response that will be returned to the client
- * @param uri the URI
- * @param method the HTTP Method
- *
- * @deprecated this method exists only until we can remove the Cluster Manager from the picture all together. It will then be removed.
- */
- @Deprecated
- private void mergeNCMBulletins(final NodeResponse clientResponse, final URI uri, final String method) {
- // determine if we have at least one response
- final boolean hasClientResponse = clientResponse != null;
- final boolean hasSuccessfulClientResponse = hasClientResponse && clientResponse.is2xx();
-
- if (hasSuccessfulClientResponse && clusterManager.isControllerStatusEndpoint(uri, method)) {
- // for now, we need to merge the NCM's bulletins too.
- final ControllerStatusEntity responseEntity = (ControllerStatusEntity) clientResponse.getUpdatedEntity();
- final ControllerStatusDTO mergedStatus = responseEntity.getControllerStatus();
-
- final int totalNodeCount = clusterManager.getNodeIds().size();
- final int connectedNodeCount = clusterManager.getNodeIds(Status.CONNECTED).size();
-
- final List ncmControllerBulletins = clusterManager.getBulletinRepository().findBulletinsForController();
- mergedStatus.setBulletins(clusterManager.mergeNCMBulletins(mergedStatus.getBulletins(), ncmControllerBulletins));
-
- // get the controller service bulletins
- final BulletinQuery controllerServiceQuery = new BulletinQuery.Builder().sourceType(ComponentType.CONTROLLER_SERVICE).build();
- final List ncmServiceBulletins = clusterManager.getBulletinRepository().findBulletins(controllerServiceQuery);
- mergedStatus.setControllerServiceBulletins(clusterManager.mergeNCMBulletins(mergedStatus.getControllerServiceBulletins(), ncmServiceBulletins));
-
- // get the reporting task bulletins
- final BulletinQuery reportingTaskQuery = new BulletinQuery.Builder().sourceType(ComponentType.REPORTING_TASK).build();
- final List ncmReportingTaskBulletins = clusterManager.getBulletinRepository().findBulletins(reportingTaskQuery);
- mergedStatus.setReportingTaskBulletins(clusterManager.mergeNCMBulletins(mergedStatus.getReportingTaskBulletins(), ncmReportingTaskBulletins));
-
- mergedStatus.setConnectedNodeCount(connectedNodeCount);
- mergedStatus.setTotalNodeCount(totalNodeCount);
- StatusMerger.updatePrettyPrintedFields(mergedStatus);
- }
- }
-
@Override
public Set getProblematicNodeResponses(final Set allResponses) {
@@ -225,7 +163,7 @@ public class StandardHttpResponseMerger implements HttpResponseMerger {
responses.stream()
.parallel() // parallelize the draining of the responses, since we have multiple streams to consume
.filter(response -> response != exclude) // don't include the explicitly excluded node
- .filter(response -> response.getStatus() != NODE_CONTINUE_STATUS_CODE) // don't include any 150-NodeContinue responses because they contain no content
+ .filter(response -> response.getStatus() != RequestReplicator.NODE_CONTINUE_STATUS_CODE) // don't include any 150-NodeContinue responses because they contain no content
.forEach(response -> drainResponse(response)); // drain all node responses that didn't get filtered out
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/AbstractNodeStatusEndpoint.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/AbstractNodeStatusEndpoint.java
index cc73e3423a..f930bf1a22 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/AbstractNodeStatusEndpoint.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/AbstractNodeStatusEndpoint.java
@@ -24,11 +24,16 @@ import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.web.api.entity.Entity;
-public abstract class AbstractNodeStatusEndpoint extends AbstractSingleEntityEndpoint {
+public abstract class AbstractNodeStatusEndpoint extends AbstractSingleDTOEndpoint {
@Override
protected final void mergeResponses(DtoType clientDto, Map dtoMap, Set successfulResponses, Set problematicResponses) {
- final NodeIdentifier selectedNodeId = dtoMap.entrySet().stream().filter(e -> e.getValue() == clientDto).map(e -> e.getKey()).findFirst().orElse(null);
+ final NodeIdentifier selectedNodeId = dtoMap.entrySet().stream()
+ .filter(e -> e.getValue() == clientDto)
+ .map(e -> e.getKey())
+ .findFirst()
+ .orElse(null);
+
if (selectedNodeId == null) {
throw new IllegalArgumentException("Attempted to merge Status request but could not find the appropriate Node Identifier");
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/AbstractSingleDTOEndpoint.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/AbstractSingleDTOEndpoint.java
new file mode 100644
index 0000000000..db3cfd188b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/AbstractSingleDTOEndpoint.java
@@ -0,0 +1,115 @@
+/*
+ * 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.http.endpoints;
+
+import java.net.URI;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
+import org.apache.nifi.cluster.manager.NodeResponse;
+import org.apache.nifi.cluster.protocol.NodeIdentifier;
+import org.apache.nifi.web.api.entity.Entity;
+
+public abstract class AbstractSingleDTOEndpoint implements EndpointResponseMerger {
+
+ @Override
+ public final NodeResponse merge(final URI uri, final String method, final Set successfulResponses, final Set problematicResponses, final NodeResponse clientResponse) {
+ if (!canHandle(uri, method)) {
+ throw new IllegalArgumentException("Cannot use Endpoint Mapper of type " + getClass().getSimpleName() + " to map responses for URI " + uri + ", HTTP Method " + method);
+ }
+
+ final EntityType responseEntity = clientResponse.getClientResponse().getEntity(getEntityClass());
+ final DtoType dto = getDto(responseEntity);
+
+ final Map dtoMap = new HashMap<>();
+ for (final NodeResponse nodeResponse : successfulResponses) {
+ final EntityType nodeResponseEntity = nodeResponse == clientResponse ? responseEntity : nodeResponse.getClientResponse().getEntity(getEntityClass());
+ final DtoType nodeDto = getDto(nodeResponseEntity);
+ dtoMap.put(nodeResponse.getNodeId(), nodeDto);
+ }
+
+ mergeResponses(dto, dtoMap, successfulResponses, problematicResponses);
+ return new NodeResponse(clientResponse, responseEntity);
+ }
+
+
+ /**
+ * Normalizes the validation errors by prepending the corresponding nodes when the error does not exist across all nodes.
+ *
+ * @param validationErrorMap map
+ * @param totalNodes total
+ * @return normalized errors
+ */
+ protected Set normalizedMergedValidationErrors(final Map> validationErrorMap, int totalNodes) {
+ final Set normalizedValidationErrors = new HashSet<>();
+ for (final Map.Entry> validationEntry : validationErrorMap.entrySet()) {
+ final String msg = validationEntry.getKey();
+ final Set nodeIds = validationEntry.getValue();
+
+ if (nodeIds.size() == totalNodes) {
+ normalizedValidationErrors.add(msg);
+ } else {
+ nodeIds.forEach(id -> normalizedValidationErrors.add(id.getApiAddress() + ":" + id.getApiPort() + " -- " + msg));
+ }
+ }
+ return normalizedValidationErrors;
+ }
+
+ /**
+ * Merges the validation errors into the specified map, recording the corresponding node identifier.
+ *
+ * @param validationErrorMap map
+ * @param nodeId id
+ * @param nodeValidationErrors errors
+ */
+ protected void mergeValidationErrors(final Map> validationErrorMap, final NodeIdentifier nodeId, final Collection nodeValidationErrors) {
+ if (nodeValidationErrors != null) {
+ nodeValidationErrors.stream().forEach(
+ err -> validationErrorMap.computeIfAbsent(err, k -> new HashSet())
+ .add(nodeId));
+ }
+ }
+
+ /**
+ * @return the class that represents the type of Entity that is expected by this response mapper
+ */
+ protected abstract Class getEntityClass();
+
+ /**
+ * Extracts the DTO from the given entity
+ *
+ * @param entity the entity to extract the DTO from
+ * @return the DTO from the given entity
+ */
+ protected abstract DtoType getDto(EntityType entity);
+
+ /**
+ * Merges the responses from all nodes in the given map into the single given DTO
+ *
+ * @param clientDto the DTO to merge responses into
+ * @param dtoMap the responses from all nodes
+ * @param successfulResponses the responses from nodes that completed the request successfully
+ * @param problematicResponses the responses from nodes that did not complete the request successfully
+ */
+ protected abstract void mergeResponses(DtoType clientDto, Map dtoMap, Set successfulResponses, Set problematicResponses);
+
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/AbstractSingleEntityEndpoint.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/AbstractSingleEntityEndpoint.java
index 66db949f14..026ececcf3 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/AbstractSingleEntityEndpoint.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/AbstractSingleEntityEndpoint.java
@@ -29,8 +29,7 @@ import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.web.api.entity.Entity;
-public abstract class AbstractSingleEntityEndpoint implements EndpointResponseMerger {
-
+public abstract class AbstractSingleEntityEndpoint implements EndpointResponseMerger {
@Override
public final NodeResponse merge(final URI uri, final String method, final Set successfulResponses, final Set problematicResponses, final NodeResponse clientResponse) {
if (!canHandle(uri, method)) {
@@ -38,19 +37,31 @@ public abstract class AbstractSingleEntityEndpoint dtoMap = new HashMap<>();
+ final Map entityMap = new HashMap<>();
for (final NodeResponse nodeResponse : successfulResponses) {
final EntityType nodeResponseEntity = nodeResponse == clientResponse ? responseEntity : nodeResponse.getClientResponse().getEntity(getEntityClass());
- final DtoType nodeDto = getDto(nodeResponseEntity);
- dtoMap.put(nodeResponse.getNodeId(), nodeDto);
+ entityMap.put(nodeResponse.getNodeId(), nodeResponseEntity);
}
- mergeResponses(dto, dtoMap, successfulResponses, problematicResponses);
+ mergeResponses(responseEntity, entityMap, successfulResponses, problematicResponses);
return new NodeResponse(clientResponse, responseEntity);
}
+ /**
+ * Merges the validation errors into the specified map, recording the corresponding node identifier.
+ *
+ * @param validationErrorMap map
+ * @param nodeId id
+ * @param nodeValidationErrors errors
+ */
+ protected void mergeValidationErrors(final Map> validationErrorMap, final NodeIdentifier nodeId, final Collection nodeValidationErrors) {
+ if (nodeValidationErrors != null) {
+ nodeValidationErrors.stream().forEach(
+ err -> validationErrorMap.computeIfAbsent(err, k -> new HashSet())
+ .add(nodeId));
+ }
+ }
/**
* Normalizes the validation errors by prepending the corresponding nodes when the error does not exist across all nodes.
@@ -74,42 +85,19 @@ public abstract class AbstractSingleEntityEndpoint> validationErrorMap, final NodeIdentifier nodeId, final Collection nodeValidationErrors) {
- if (nodeValidationErrors != null) {
- nodeValidationErrors.stream().forEach(
- err -> validationErrorMap.computeIfAbsent(err, k -> new HashSet())
- .add(nodeId));
- }
- }
-
/**
* @return the class that represents the type of Entity that is expected by this response mapper
*/
protected abstract Class getEntityClass();
/**
- * Extracts the DTO from the given entity
+ * Merges the responses from all nodes in the given map into the single given entity
*
- * @param entity the entity to extract the DTO from
- * @return the DTO from the given entity
- */
- protected abstract DtoType getDto(EntityType entity);
-
- /**
- * Merges the responses from all nodes in the given map into the single given DTO
- *
- * @param clientDto the DTO to merge responses into
- * @param dtoMap the responses from all nodes
+ * @param clientEntity the Entity to merge responses into
+ * @param entityMap the responses from all nodes
* @param successfulResponses the responses from nodes that completed the request successfully
* @param problematicResponses the responses from nodes that did not complete the request successfully
*/
- protected abstract void mergeResponses(DtoType clientDto, Map dtoMap, Set successfulResponses, Set problematicResponses);
+ protected abstract void mergeResponses(EntityType clientEntity, Map entityMap, Set successfulResponses, Set problematicResponses);
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/BulletinBoardEndpointMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/BulletinBoardEndpointMerger.java
index 799d2795c1..8da6353ae5 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/BulletinBoardEndpointMerger.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/BulletinBoardEndpointMerger.java
@@ -32,7 +32,7 @@ import org.apache.nifi.web.api.dto.BulletinBoardDTO;
import org.apache.nifi.web.api.dto.BulletinDTO;
import org.apache.nifi.web.api.entity.BulletinBoardEntity;
-public class BulletinBoardEndpointMerger extends AbstractSingleEntityEndpoint {
+public class BulletinBoardEndpointMerger extends AbstractSingleDTOEndpoint {
public static final Pattern BULLETIN_BOARD_URI_PATTERN = Pattern.compile("/nifi-api/flow/bulletin-board");
@Override
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ComponentStateEndpointMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ComponentStateEndpointMerger.java
index 0598259e8c..9976de2620 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ComponentStateEndpointMerger.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ComponentStateEndpointMerger.java
@@ -33,7 +33,7 @@ import org.apache.nifi.web.api.dto.StateEntryDTO;
import org.apache.nifi.web.api.dto.StateMapDTO;
import org.apache.nifi.web.api.entity.ComponentStateEntity;
-public class ComponentStateEndpointMerger extends AbstractSingleEntityEndpoint {
+public class ComponentStateEndpointMerger extends AbstractSingleDTOEndpoint {
public static final Pattern PROCESSOR_STATE_URI_PATTERN = Pattern.compile("/nifi-api/processors/[a-f0-9\\-]{36}/state");
public static final Pattern CONTROLLER_SERVICE_STATE_URI_PATTERN = Pattern.compile("/nifi-api/controller-services/node/[a-f0-9\\-]{36}/state");
public static final Pattern REPORTING_TASK_STATE_URI_PATTERN = Pattern.compile("/nifi-api/reporting-tasks/node/[a-f0-9\\-]{36}/state");
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ControllerServiceEndpointMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ControllerServiceEndpointMerger.java
index 448b44e1eb..10fa28196f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ControllerServiceEndpointMerger.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ControllerServiceEndpointMerger.java
@@ -31,7 +31,7 @@ import java.util.Map;
import java.util.Set;
import java.util.regex.Pattern;
-public class ControllerServiceEndpointMerger extends AbstractSingleEntityEndpoint {
+public class ControllerServiceEndpointMerger extends AbstractSingleDTOEndpoint {
public static final String CONTROLLER_SERVICES_URI = "/nifi-api/controller-services/node";
public static final Pattern CONTROLLER_SERVICE_URI_PATTERN = Pattern.compile("/nifi-api/controller-services/node/[a-f0-9\\-]{36}");
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ControllerStatusEndpointMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ControllerStatusEndpointMerger.java
index 50514f8d25..30671db686 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ControllerStatusEndpointMerger.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ControllerStatusEndpointMerger.java
@@ -29,7 +29,7 @@ import org.apache.nifi.web.api.dto.BulletinDTO;
import org.apache.nifi.web.api.dto.status.ControllerStatusDTO;
import org.apache.nifi.web.api.entity.ControllerStatusEntity;
-public class ControllerStatusEndpointMerger extends AbstractSingleEntityEndpoint {
+public class ControllerStatusEndpointMerger extends AbstractSingleDTOEndpoint {
public static final Pattern CONTROLLER_STATUS_URI_PATTERN = Pattern.compile("/nifi-api/flow/status");
@Override
@@ -71,13 +71,6 @@ public class ControllerStatusEndpointMerger extends AbstractSingleEntityEndpoint
StatusMerger.merge(mergedStatus, nodeStatus);
}
-
- final int totalNodeCount = successfulResponses.size() + problematicResponses.size();
- final int connectedNodeCount = successfulResponses.size(); // all nodes that responded successfully must be connected. Those that did not will be disconnected.
-
- mergedStatus.setConnectedNodeCount(connectedNodeCount);
- mergedStatus.setTotalNodeCount(totalNodeCount);
- StatusMerger.updatePrettyPrintedFields(mergedStatus);
}
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/DropRequestEndpiontMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/DropRequestEndpiontMerger.java
index f6025b8749..f78d6287b3 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/DropRequestEndpiontMerger.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/DropRequestEndpiontMerger.java
@@ -29,7 +29,7 @@ import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.web.api.dto.DropRequestDTO;
import org.apache.nifi.web.api.entity.DropRequestEntity;
-public class DropRequestEndpiontMerger extends AbstractSingleEntityEndpoint {
+public class DropRequestEndpiontMerger extends AbstractSingleDTOEndpoint {
public static final Pattern DROP_REQUESTS_URI = Pattern.compile("/nifi-api/flowfile-queues/[a-f0-9\\-]{36}/drop-requests");
public static final Pattern DROP_REQUEST_URI = Pattern.compile("/nifi-api/flowfile-queues/[a-f0-9\\-]{36}/drop-requests/[a-f0-9\\-]{36}");
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/FlowMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/FlowMerger.java
new file mode 100644
index 0000000000..c033d19297
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/FlowMerger.java
@@ -0,0 +1,267 @@
+/*
+ * 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.http.endpoints;
+
+import org.apache.nifi.cluster.manager.NodeResponse;
+import org.apache.nifi.cluster.manager.StatusMerger;
+import org.apache.nifi.cluster.protocol.NodeIdentifier;
+import org.apache.nifi.web.api.dto.flow.FlowDTO;
+import org.apache.nifi.web.api.dto.flow.ProcessGroupFlowDTO;
+import org.apache.nifi.web.api.dto.status.ConnectionStatusDTO;
+import org.apache.nifi.web.api.dto.status.PortStatusDTO;
+import org.apache.nifi.web.api.dto.status.ProcessGroupStatusDTO;
+import org.apache.nifi.web.api.dto.status.ProcessorStatusDTO;
+import org.apache.nifi.web.api.dto.status.RemoteProcessGroupStatusDTO;
+import org.apache.nifi.web.api.entity.ConnectionEntity;
+import org.apache.nifi.web.api.entity.FunnelEntity;
+import org.apache.nifi.web.api.entity.LabelEntity;
+import org.apache.nifi.web.api.entity.PortEntity;
+import org.apache.nifi.web.api.entity.ProcessGroupEntity;
+import org.apache.nifi.web.api.entity.ProcessGroupFlowEntity;
+import org.apache.nifi.web.api.entity.ProcessorEntity;
+import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
+
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+public class FlowMerger extends AbstractSingleDTOEndpoint {
+ public static final Pattern FLOW_URI_PATTERN = Pattern.compile("/nifi-api/flow/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))");
+
+ @Override
+ public boolean canHandle(final URI uri, final String method) {
+ return "GET".equalsIgnoreCase(method) && FLOW_URI_PATTERN.matcher(uri.getPath()).matches();
+ }
+
+ @Override
+ protected Class getEntityClass() {
+ return ProcessGroupFlowEntity.class;
+ }
+
+ @Override
+ protected ProcessGroupFlowDTO getDto(final ProcessGroupFlowEntity entity) {
+ return entity.getProcessGroupFlow();
+ }
+
+ @Override
+ protected void mergeResponses(final ProcessGroupFlowDTO clientDto, final Map dtoMap,
+ final Set successfulResponses, final Set problematicResponses) {
+
+ final FlowDTO flowDto = clientDto.getFlow();
+
+ final Map> connections = new HashMap<>();
+ final Map> funnels = new HashMap<>();
+ final Map> inputPorts = new HashMap<>();
+ final Map> labels = new HashMap<>();
+ final Map> outputPorts = new HashMap<>();
+ final Map> processors = new HashMap<>();
+ final Map> rpgs = new HashMap<>();
+ final Map> processGroups = new HashMap<>();
+
+ // Create mapping of ComponentID -> all components with that ID (one per node)
+ for (final ProcessGroupFlowDTO nodeGroupFlowDto : dtoMap.values()) {
+ final FlowDTO nodeFlowDto = nodeGroupFlowDto.getFlow();
+
+ // Merge connection statuses
+ for (final ConnectionEntity entity : nodeFlowDto.getConnections()) {
+ connections.computeIfAbsent(entity.getId(), id -> new ArrayList<>()).add(entity);
+ }
+
+ for (final FunnelEntity entity : nodeFlowDto.getFunnels()) {
+ funnels.computeIfAbsent(entity.getId(), id -> new ArrayList<>()).add(entity);
+ }
+
+ for (final PortEntity entity : nodeFlowDto.getInputPorts()) {
+ inputPorts.computeIfAbsent(entity.getId(), id -> new ArrayList<>()).add(entity);
+ }
+
+ for (final PortEntity entity : nodeFlowDto.getOutputPorts()) {
+ outputPorts.computeIfAbsent(entity.getId(), id -> new ArrayList<>()).add(entity);
+ }
+
+ for (final LabelEntity entity : nodeFlowDto.getLabels()) {
+ labels.computeIfAbsent(entity.getId(), id -> new ArrayList<>()).add(entity);
+ }
+
+ for (final ProcessorEntity entity : nodeFlowDto.getProcessors()) {
+ processors.computeIfAbsent(entity.getId(), id -> new ArrayList<>()).add(entity);
+ }
+
+ for (final RemoteProcessGroupEntity entity : nodeFlowDto.getRemoteProcessGroups()) {
+ rpgs.computeIfAbsent(entity.getId(), id -> new ArrayList<>()).add(entity);
+ }
+
+ for (final ProcessGroupEntity entity : nodeFlowDto.getProcessGroups()) {
+ processGroups.computeIfAbsent(entity.getId(), id -> new ArrayList<>()).add(entity);
+ }
+ }
+
+ //
+ // Merge the components that are grouped together by ID
+ //
+
+ // Merge connections
+ final Set mergedConnections = new HashSet<>();
+ for (final List connectionList : connections.values()) {
+ mergedConnections.add(mergeConnections(connectionList));
+ }
+ flowDto.setConnections(mergedConnections);
+
+ // Merge funnel statuses
+ final Set mergedFunnels = new HashSet<>();
+ for (final List funnelList : funnels.values()) {
+ mergedFunnels.add(mergeFunnels(funnelList));
+ }
+ flowDto.setFunnels(mergedFunnels);
+
+ // Merge input ports
+ final Set mergedInputPorts = new HashSet<>();
+ for (final List portList : inputPorts.values()) {
+ mergedInputPorts.add(mergePorts(portList));
+ }
+ flowDto.setInputPorts(mergedInputPorts);
+
+ // Merge output ports
+ final Set mergedOutputPorts = new HashSet<>();
+ for (final List portList : outputPorts.values()) {
+ mergedOutputPorts.add(mergePorts(portList));
+ }
+ flowDto.setOutputPorts(mergedOutputPorts);
+
+ // Merge labels
+ final Set mergedLabels = new HashSet<>();
+ for (final List labelList : labels.values()) {
+ mergedLabels.add(mergeLabels(labelList));
+ }
+ flowDto.setLabels(mergedLabels);
+
+
+ // Merge processors
+ final Set mergedProcessors = new HashSet<>();
+ for (final List processorList : processors.values()) {
+ mergedProcessors.add(mergeProcessors(processorList));
+ }
+ flowDto.setProcessors(mergedProcessors);
+
+
+ // Merge Remote Process Groups
+ final Set mergedRpgs = new HashSet<>();
+ for (final List rpgList : rpgs.values()) {
+ mergedRpgs.add(mergeRemoteProcessGroups(rpgList));
+ }
+ flowDto.setRemoteProcessGroups(mergedRpgs);
+
+
+ // Merge Process Groups
+ final Set mergedGroups = new HashSet<>();
+ for (final List groupList : processGroups.values()) {
+ mergedGroups.add(mergeProcessGroups(groupList));
+ }
+ flowDto.setProcessGroups(mergedGroups);
+ }
+
+ private ConnectionEntity mergeConnections(final List connections) {
+ final ConnectionEntity merged = connections.get(0);
+ final ConnectionStatusDTO statusDto = merged.getStatus();
+ statusDto.setNodeSnapshots(null);
+
+ for (final ConnectionEntity entity : connections) {
+ if (entity != merged) {
+ StatusMerger.merge(merged.getStatus().getAggregateSnapshot(), entity.getStatus().getAggregateSnapshot());
+ }
+ }
+
+ return merged;
+ }
+
+ private PortEntity mergePorts(final List ports) {
+ final PortEntity merged = ports.get(0);
+ final PortStatusDTO statusDto = merged.getStatus();
+ statusDto.setNodeSnapshots(null);
+
+ for (final PortEntity entity : ports) {
+ if (entity != merged) {
+ StatusMerger.merge(merged.getStatus().getAggregateSnapshot(), entity.getStatus().getAggregateSnapshot());
+ }
+ }
+
+ return merged;
+ }
+
+ private FunnelEntity mergeFunnels(final List funnels) {
+ return funnels.get(0);
+ }
+
+ private LabelEntity mergeLabels(final List labels) {
+ return labels.get(0);
+ }
+
+ private ProcessorEntity mergeProcessors(final List processors) {
+ final ProcessorEntity merged = processors.get(0);
+ final ProcessorStatusDTO statusDto = merged.getStatus();
+ statusDto.setNodeSnapshots(null);
+
+ for (final ProcessorEntity entity : processors) {
+ if (entity != merged) {
+ StatusMerger.merge(merged.getStatus().getAggregateSnapshot(), entity.getStatus().getAggregateSnapshot());
+ }
+ }
+
+ return merged;
+ }
+
+
+ private RemoteProcessGroupEntity mergeRemoteProcessGroups(final List rpgs) {
+ final RemoteProcessGroupEntity merged = rpgs.get(0);
+ final RemoteProcessGroupStatusDTO statusDto = merged.getStatus();
+ statusDto.setNodeSnapshots(null);
+
+ for (final RemoteProcessGroupEntity entity : rpgs) {
+ if (entity != merged) {
+ StatusMerger.merge(merged.getStatus().getAggregateSnapshot(), entity.getStatus().getAggregateSnapshot());
+ }
+ }
+
+ return merged;
+ }
+
+ private ProcessGroupEntity mergeProcessGroups(final List groups) {
+ final ProcessGroupEntity merged = groups.get(0);
+ final ProcessGroupStatusDTO statusDto = merged.getStatus();
+ statusDto.setNodeSnapshots(null);
+
+ for (final ProcessGroupEntity entity : groups) {
+ if (entity != merged) {
+ StatusMerger.merge(merged.getStatus().getAggregateSnapshot(), entity.getStatus().getAggregateSnapshot());
+ }
+ }
+
+ // We merge only the statuses of the Process Groups. The child components are not
+ // necessary for a FlowProcessGroupDTO, so we just ensure that they are null
+ if (merged.getComponent() != null) {
+ merged.getComponent().setContents(null);
+ }
+
+ return merged;
+ }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/FlowSnippetEndpointMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/FlowSnippetEndpointMerger.java
index 2063de4d49..78ccad6ea7 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/FlowSnippetEndpointMerger.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/FlowSnippetEndpointMerger.java
@@ -57,22 +57,12 @@ public class FlowSnippetEndpointMerger implements EndpointResponseMerger {
final FlowDTO nodeContents = nodeResponseEntity.getFlow();
for (final ProcessorEntity nodeProcessor : nodeContents.getProcessors()) {
- Map innerMap = processorMap.get(nodeProcessor.getId());
- if (innerMap == null) {
- innerMap = new HashMap<>();
- processorMap.put(nodeProcessor.getId(), innerMap);
- }
-
+ Map innerMap = processorMap.computeIfAbsent(nodeProcessor.getId(), id -> new HashMap<>());
innerMap.put(nodeResponse.getNodeId(), nodeProcessor);
}
for (final RemoteProcessGroupEntity nodeRemoteProcessGroup : nodeContents.getRemoteProcessGroups()) {
- Map innerMap = remoteProcessGroupMap.get(nodeRemoteProcessGroup.getId());
- if (innerMap == null) {
- innerMap = new HashMap<>();
- remoteProcessGroupMap.put(nodeRemoteProcessGroup.getId(), innerMap);
- }
-
+ Map innerMap = remoteProcessGroupMap.computeIfAbsent(nodeRemoteProcessGroup.getId(), id -> new HashMap<>());
innerMap.put(nodeResponse.getNodeId(), nodeRemoteProcessGroup);
}
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ListFlowFilesEndpointMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ListFlowFilesEndpointMerger.java
index 415334e9b5..2d17c4946e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ListFlowFilesEndpointMerger.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ListFlowFilesEndpointMerger.java
@@ -35,7 +35,7 @@ import org.apache.nifi.web.api.dto.ListingRequestDTO;
import org.apache.nifi.web.api.dto.QueueSizeDTO;
import org.apache.nifi.web.api.entity.ListingRequestEntity;
-public class ListFlowFilesEndpointMerger extends AbstractSingleEntityEndpoint {
+public class ListFlowFilesEndpointMerger extends AbstractSingleDTOEndpoint {
public static final Pattern LISTING_REQUESTS_URI = Pattern.compile("/nifi-api/flowfile-queues/[a-f0-9\\-]{36}/listing-requests");
public static final Pattern LISTING_REQUEST_URI = Pattern.compile("/nifi-api/flowfile-queues/[a-f0-9\\-]{36}/listing-requests/[a-f0-9\\-]{36}");
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ProcessorEndpointMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ProcessorEndpointMerger.java
index a892e8ad63..bd50eca407 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ProcessorEndpointMerger.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ProcessorEndpointMerger.java
@@ -25,11 +25,12 @@ import java.util.regex.Pattern;
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
import org.apache.nifi.cluster.manager.NodeResponse;
+import org.apache.nifi.cluster.manager.StatusMerger;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.web.api.dto.ProcessorDTO;
import org.apache.nifi.web.api.entity.ProcessorEntity;
-public class ProcessorEndpointMerger extends AbstractSingleEntityEndpoint implements EndpointResponseMerger {
+public class ProcessorEndpointMerger extends AbstractSingleEntityEndpoint implements EndpointResponseMerger {
public static final Pattern PROCESSORS_URI_PATTERN = Pattern.compile("/nifi-api/processors");
public static final Pattern PROCESSOR_URI_PATTERN = Pattern.compile("/nifi-api/processors/[a-f0-9\\-]{36}");
public static final Pattern CLUSTER_PROCESSOR_URI_PATTERN = Pattern.compile("/nifi-api/cluster/processors/[a-f0-9\\-]{36}");
@@ -51,12 +52,7 @@ public class ProcessorEndpointMerger extends AbstractSingleEntityEndpoint dtoMap, final Set successfulResponses,
final Set problematicResponses) {
final Map> validationErrorMap = new HashMap<>();
@@ -73,6 +69,7 @@ public class ProcessorEndpointMerger extends AbstractSingleEntityEndpoint entityMap, final Set successfulResponses,
final Set problematicResponses) {
@@ -84,6 +81,14 @@ public class ProcessorEndpointMerger extends AbstractSingleEntityEndpoint entry : entityMap.entrySet()) {
+ final NodeIdentifier nodeId = entry.getKey();
+ final ProcessorEntity entity = entry.getValue();
+ if (entity != clientEntity) {
+ StatusMerger.merge(clientEntity.getStatus(), entity.getStatus(), nodeId.getId(), nodeId.getApiAddress(), nodeId.getApiPort());
+ }
+ }
+
mergeResponses(clientDto, dtoMap, successfulResponses, problematicResponses);
}
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ProvenanceEventEndpointMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ProvenanceEventEndpointMerger.java
index 3f895bd8ab..c05ca7ffee 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ProvenanceEventEndpointMerger.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ProvenanceEventEndpointMerger.java
@@ -27,7 +27,7 @@ import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.web.api.dto.provenance.ProvenanceEventDTO;
import org.apache.nifi.web.api.entity.ProvenanceEventEntity;
-public class ProvenanceEventEndpointMerger extends AbstractSingleEntityEndpoint {
+public class ProvenanceEventEndpointMerger extends AbstractSingleDTOEndpoint {
public static final Pattern PROVENANCE_EVENT_URI = Pattern.compile("/nifi-api/provenance/events/[0-9]+");
@Override
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/RemoteProcessGroupEndpointMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/RemoteProcessGroupEndpointMerger.java
index 56636fbe62..732d5274b3 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/RemoteProcessGroupEndpointMerger.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/RemoteProcessGroupEndpointMerger.java
@@ -33,7 +33,7 @@ import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO;
import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
-public class RemoteProcessGroupEndpointMerger extends AbstractSingleEntityEndpoint {
+public class RemoteProcessGroupEndpointMerger extends AbstractSingleDTOEndpoint {
public static final Pattern REMOTE_PROCESS_GROUPS_URI_PATTERN = Pattern.compile("/nifi-api/remote-process-groups");
public static final Pattern REMOTE_PROCESS_GROUP_URI_PATTERN = Pattern.compile("/nifi-api/remote-process-groups/[a-f0-9\\-]{36}");
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ReportingTaskEndpointMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ReportingTaskEndpointMerger.java
index b45d8ecf31..245d3bda87 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ReportingTaskEndpointMerger.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ReportingTaskEndpointMerger.java
@@ -28,7 +28,7 @@ import java.util.Map;
import java.util.Set;
import java.util.regex.Pattern;
-public class ReportingTaskEndpointMerger extends AbstractSingleEntityEndpoint {
+public class ReportingTaskEndpointMerger extends AbstractSingleDTOEndpoint {
public static final String REPORTING_TASKS_URI = "/nifi-api/controller/reporting-tasks/node";
public static final Pattern REPORTING_TASK_URI_PATTERN = Pattern.compile("/nifi-api/reporting-tasks/node/[a-f0-9\\-]{36}");
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/RequestReplicator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/RequestReplicator.java
index b64c7660d9..a4b762ae1a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/RequestReplicator.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/RequestReplicator.java
@@ -27,6 +27,7 @@ public interface RequestReplicator {
public static final String REQUEST_TRANSACTION_ID_HEADER = "X-RequestTransactionId";
public static final String CLUSTER_ID_GENERATION_SEED_HEADER = "X-Cluster-Id-Generation-Seed";
+ public static final String REPLICATION_INDICATOR_HEADER = "X-Request-Replicated";
/**
* The HTTP header that the requestor specifies to ask a node if they are able to process a given request. The value
@@ -37,21 +38,30 @@ public interface RequestReplicator {
public static final String NODE_CONTINUE = "150-NodeContinue";
public static final int NODE_CONTINUE_STATUS_CODE = 150;
-
- /**
- * Starts the instance for replicating requests. Calling this method on an already started instance has no effect.
- */
- void start();
+ public static final String CLAIM_CANCEL_HEADER = "X-Cancel-Claim";
/**
* Stops the instance from replicating requests. Calling this method on a stopped instance has no effect.
*/
- void stop();
+ void shutdown();
+
/**
- * @return true if the instance is started; false otherwise.
+ * Replicates a request to each node in the cluster. If the request attempts to modify the flow and there is a node
+ * that is not currently connected, an Exception will be thrown. Otherwise, the returned AsyncClusterResponse object
+ * will contain the results that are immediately available, as well as an identifier for obtaining an updated result
+ * later.
+ *
+ * @param method the HTTP method (e.g., POST, PUT)
+ * @param uri the base request URI (up to, but not including, the query string)
+ * @param entity an entity
+ * @param headers any HTTP headers
+ * @return an AsyncClusterResponse that indicates the current status of the request and provides an identifier for obtaining an updated response later
+ *
+ * @throws ConnectingNodeMutableRequestException if the request attempts to modify the flow and there is a node that is in the CONNECTING state
+ * @throws DisconnectedNodeMutableRequestException if the request attempts to modify the flow and there is a node that is in the DISCONNECTED state
*/
- boolean isRunning();
+ AsyncClusterResponse replicate(String method, URI uri, Object entity, Map headers);
/**
* Requests are sent to each node in the given set of Node Identifiers. The returned AsyncClusterResponse object will contain
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java
index dd4d2cedbf..ed83159d3e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java
@@ -47,14 +47,14 @@ import org.apache.nifi.cluster.coordination.ClusterCoordinator;
import org.apache.nifi.cluster.coordination.http.HttpResponseMerger;
import org.apache.nifi.cluster.coordination.http.StandardHttpResponseMerger;
import org.apache.nifi.cluster.coordination.node.NodeConnectionState;
-import org.apache.nifi.cluster.flow.DataFlowManagementService;
-import org.apache.nifi.cluster.flow.PersistedFlowState;
+import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.manager.exception.ConnectingNodeMutableRequestException;
import org.apache.nifi.cluster.manager.exception.DisconnectedNodeMutableRequestException;
import org.apache.nifi.cluster.manager.exception.IllegalClusterStateException;
+import org.apache.nifi.cluster.manager.exception.NoConnectedNodesException;
+import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
import org.apache.nifi.cluster.manager.exception.UriConstructionException;
-import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.events.EventReporter;
import org.apache.nifi.logging.NiFiLog;
@@ -76,14 +76,12 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
private static final int MAX_CONCURRENT_REQUESTS = 100;
private final Client client; // the client to use for issuing requests
- private final int numThreads; // number of threads to use for request replication
private final int connectionTimeoutMs; // connection timeout per node request
private final int readTimeoutMs; // read timeout per node request
private final HttpResponseMerger responseMerger;
private final EventReporter eventReporter;
private final RequestCompletionCallback callback;
private final ClusterCoordinator clusterCoordinator;
- private final DataFlowManagementService dfmService;
private ExecutorService executorService;
private ScheduledExecutorService maintenanceExecutor;
@@ -101,8 +99,8 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
* @param eventReporter an EventReporter that can be used to notify users of interesting events. May be null.
*/
public ThreadPoolRequestReplicator(final int numThreads, final Client client, final ClusterCoordinator clusterCoordinator,
- final RequestCompletionCallback callback, final EventReporter eventReporter, final DataFlowManagementService dfmService) {
- this(numThreads, client, clusterCoordinator, "3 sec", "3 sec", callback, eventReporter, null, dfmService);
+ final RequestCompletionCallback callback, final EventReporter eventReporter) {
+ this(numThreads, client, clusterCoordinator, "5 sec", "5 sec", callback, eventReporter);
}
/**
@@ -117,36 +115,33 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
* @param eventReporter an EventReporter that can be used to notify users of interesting events. May be null.
*/
public ThreadPoolRequestReplicator(final int numThreads, final Client client, final ClusterCoordinator clusterCoordinator,
- final String connectionTimeout, final String readTimeout, final RequestCompletionCallback callback, final EventReporter eventReporter,
- final WebClusterManager clusterManager, final DataFlowManagementService dfmService) {
+ final String connectionTimeout, final String readTimeout, final RequestCompletionCallback callback, final EventReporter eventReporter) {
if (numThreads <= 0) {
throw new IllegalArgumentException("The number of threads must be greater than zero.");
} else if (client == null) {
throw new IllegalArgumentException("Client may not be null.");
}
- this.numThreads = numThreads;
this.client = client;
this.clusterCoordinator = clusterCoordinator;
this.connectionTimeoutMs = (int) FormatUtils.getTimeDuration(connectionTimeout, TimeUnit.MILLISECONDS);
this.readTimeoutMs = (int) FormatUtils.getTimeDuration(readTimeout, TimeUnit.MILLISECONDS);
- this.responseMerger = new StandardHttpResponseMerger(clusterManager);
+ this.responseMerger = new StandardHttpResponseMerger();
this.eventReporter = eventReporter;
this.callback = callback;
- this.dfmService = dfmService;
client.getProperties().put(ClientConfig.PROPERTY_CONNECT_TIMEOUT, connectionTimeoutMs);
client.getProperties().put(ClientConfig.PROPERTY_READ_TIMEOUT, readTimeoutMs);
client.getProperties().put(ClientConfig.PROPERTY_FOLLOW_REDIRECTS, Boolean.TRUE);
- }
- @Override
- public void start() {
- if (isRunning()) {
- return;
- }
+ final AtomicInteger threadId = new AtomicInteger(0);
+ executorService = Executors.newFixedThreadPool(numThreads, r -> {
+ final Thread t = Executors.defaultThreadFactory().newThread(r);
+ t.setDaemon(true);
+ t.setName("Replicate Request Thread-" + threadId.incrementAndGet());
+ return t;
+ });
- executorService = Executors.newFixedThreadPool(numThreads);
maintenanceExecutor = Executors.newScheduledThreadPool(1, new ThreadFactory() {
@Override
public Thread newThread(final Runnable r) {
@@ -161,25 +156,61 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
}
@Override
- public boolean isRunning() {
- return executorService != null && !executorService.isShutdown();
- }
-
- @Override
- public void stop() {
- if (!isRunning()) {
- return;
- }
-
+ public void shutdown() {
executorService.shutdown();
maintenanceExecutor.shutdown();
}
+ @Override
+ public AsyncClusterResponse replicate(String method, URI uri, Object entity, Map headers) {
+ final Map> stateMap = clusterCoordinator.getConnectionStates();
+ final boolean mutable = isMutableRequest(method, uri.getPath());
+
+ // If the request is mutable, ensure that all nodes are connected.
+ if (mutable) {
+ final List disconnected = stateMap.get(NodeConnectionState.DISCONNECTED);
+ if (disconnected != null && !disconnected.isEmpty()) {
+ if (disconnected.size() == 1) {
+ throw new DisconnectedNodeMutableRequestException("Node " + disconnected.iterator().next() + " is currently disconnected");
+ } else {
+ throw new DisconnectedNodeMutableRequestException(disconnected.size() + " Nodes are currently disconnected");
+ }
+ }
+
+ final List disconnecting = stateMap.get(NodeConnectionState.DISCONNECTING);
+ if (disconnecting != null && !disconnecting.isEmpty()) {
+ if (disconnecting.size() == 1) {
+ throw new DisconnectedNodeMutableRequestException("Node " + disconnecting.iterator().next() + " is currently disconnecting");
+ } else {
+ throw new DisconnectedNodeMutableRequestException(disconnecting.size() + " Nodes are currently disconnecting");
+ }
+ }
+
+ final List connecting = stateMap.get(NodeConnectionState.CONNECTING);
+ if (connecting != null && !connecting.isEmpty()) {
+ if (connecting.size() == 1) {
+ throw new ConnectingNodeMutableRequestException("Node " + connecting.iterator().next() + " is currently connecting");
+ } else {
+ throw new ConnectingNodeMutableRequestException(connecting.size() + " Nodes are currently connecting");
+ }
+ }
+ }
+
+ final List nodeIds = stateMap.get(NodeConnectionState.CONNECTED);
+ if (nodeIds == null || nodeIds.isEmpty()) {
+ throw new NoConnectedNodesException();
+ }
+
+ final Set nodeIdSet = new HashSet<>(nodeIds);
+ return replicate(nodeIdSet, method, uri, entity, headers);
+ }
+
@Override
public AsyncClusterResponse replicate(Set nodeIds, String method, URI uri, Object entity, Map headers) {
- final Map headersPlusIdGenerationSeed = new HashMap<>(headers);
- headersPlusIdGenerationSeed.put(RequestReplicator.CLUSTER_ID_GENERATION_SEED_HEADER, UUID.randomUUID().toString());
- return replicate(nodeIds, method, uri, entity, headersPlusIdGenerationSeed, true, null);
+ final Map updatedHeaders = new HashMap<>(headers);
+ updatedHeaders.put(RequestReplicator.CLUSTER_ID_GENERATION_SEED_HEADER, UUID.randomUUID().toString());
+ updatedHeaders.put(RequestReplicator.REPLICATION_INDICATOR_HEADER, "true");
+ return replicate(nodeIds, method, uri, entity, updatedHeaders, true, null);
}
/**
@@ -209,6 +240,18 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
throw new IllegalArgumentException("Cannot replicate request to 0 nodes");
}
+ // verify all of the nodes exist and are in the proper state
+ for (final NodeIdentifier nodeId : nodeIds) {
+ final NodeConnectionStatus status = clusterCoordinator.getConnectionStatus(nodeId);
+ if (status == null) {
+ throw new UnknownNodeException("Node " + nodeId + " does not exist in this cluster");
+ }
+
+ if (status.getState() != NodeConnectionState.CONNECTED) {
+ throw new IllegalClusterStateException("Cannot replicate request to Node " + nodeId + " because the node is not connected");
+ }
+ }
+
logger.debug("Replicating request {} {} with entity {} to {}; response is {}", method, uri, entity, nodeIds, response);
// Update headers to indicate the current revision so that we can
@@ -238,7 +281,6 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
}
logger.debug("For Request ID {}, response object is {}", requestId, response);
- // setRevision(updatedHeaders);
// if mutable request, we have to do a two-phase commit where we ask each node to verify
// that the request can take place and then, if all nodes agree that it can, we can actually
@@ -259,7 +301,6 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
finalResponse.add(nodeResponse);
};
-
// replicate the request to all nodes
final Function requestFactory =
nodeId -> new NodeHttpRequest(nodeId, method, createURI(uri, nodeId), entity, updatedHeaders, nodeCompletionCallback);
@@ -309,6 +350,21 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
return;
}
+ final Thread cancelClaimThread = new Thread(new Runnable() {
+ @Override
+ public void run() {
+ logger.debug("Found {} dissenting nodes for {} {}; canceling claim request", dissentingCount, method, uri.getPath());
+ updatedHeaders.put(CLAIM_CANCEL_HEADER, "true");
+
+ final Function requestFactory =
+ nodeId -> new NodeHttpRequest(nodeId, method, createURI(uri, nodeId), entity, updatedHeaders, null);
+
+ replicateRequest(nodeIds, uri.getScheme(), uri.getPath(), requestFactory, updatedHeaders);
+ }
+ });
+ cancelClaimThread.setName("Cancel Claims");
+ cancelClaimThread.start();
+
// Add a NodeResponse for each node to the Cluster Response
// Check that all nodes responded successfully.
for (final NodeResponse response : nodeResponses) {
@@ -354,9 +410,6 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
}
};
- // notify dataflow management service that flow state is not known
- dfmService.setPersistedFlowState(PersistedFlowState.UNKNOWN);
-
// Callback function for generating a NodeHttpRequestCallable that can be used to perform the work
final Function requestFactory = nodeId -> new NodeHttpRequest(nodeId, method, createURI(uri, nodeId), entity, updatedHeaders, completionCallback);
@@ -449,11 +502,7 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
* @param requestId the ID of the request that has been consumed by the client
*/
private void onResponseConsumed(final String requestId) {
- final AsyncClusterResponse response = responseMap.remove(requestId);
-
- if (response != null && logger.isDebugEnabled()) {
- logTimingInfo(response);
- }
+ responseMap.remove(requestId);
}
/**
@@ -466,12 +515,6 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
private void onCompletedResponse(final String requestId) {
final AsyncClusterResponse response = responseMap.get(requestId);
- if (response != null) {
- if (isMutableRequest(response.getMethod(), response.getURIPath())) {
- dfmService.setPersistedFlowState(PersistedFlowState.STALE);
- }
- }
-
if (response != null && callback != null) {
try {
callback.afterRequest(response.getURIPath(), response.getMethod(), response.getCompletedNodeResponses());
@@ -482,6 +525,10 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
}
}
+ if (response != null && logger.isDebugEnabled()) {
+ logTimingInfo(response);
+ }
+
// If we have any nodes that are slow to respond, keep track of this. If the same node is slow 3 times in
// a row, log a warning to indicate that the node is responding slowly.
final Set slowResponseNodes = ResponseUtils.findLongResponseTimes(response, 1.5D);
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
new file mode 100644
index 0000000000..e893c3aacc
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/CuratorNodeProtocolSender.java
@@ -0,0 +1,116 @@
+/*
+ * 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 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.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.nifi.util.NiFiProperties;
+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 NiFiProperties 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);
+
+ final String address = new String(coordinatorAddressBytes, StandardCharsets.UTF_8);
+
+ logger.info("Determined that Cluster Coordinator is located at {}; will use this address for sending heartbeat messages", address);
+ 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);
+ }
+
+ 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 ProtocolException("No node has yet been elected Cluster Coordinator. Cannot establish connection to cluster yet.");
+ } 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/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
new file mode 100644
index 0000000000..1a8c59ed93
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java
@@ -0,0 +1,782 @@
+/*
+ * 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 org.apache.commons.collections4.queue.CircularFifoQueue;
+import org.apache.nifi.cluster.coordination.ClusterCoordinator;
+import org.apache.nifi.cluster.coordination.http.HttpResponseMerger;
+import org.apache.nifi.cluster.coordination.http.StandardHttpResponseMerger;
+import org.apache.nifi.cluster.coordination.http.replication.RequestCompletionCallback;
+import org.apache.nifi.cluster.event.Event;
+import org.apache.nifi.cluster.event.NodeEvent;
+import org.apache.nifi.cluster.firewall.ClusterNodeFirewall;
+import org.apache.nifi.cluster.manager.NodeResponse;
+import org.apache.nifi.cluster.protocol.ComponentRevision;
+import org.apache.nifi.cluster.protocol.ConnectionRequest;
+import org.apache.nifi.cluster.protocol.ConnectionResponse;
+import org.apache.nifi.cluster.protocol.DataFlow;
+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.StandardDataFlow;
+import org.apache.nifi.cluster.protocol.impl.ClusterCoordinationProtocolSenderListener;
+import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage;
+import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage;
+import org.apache.nifi.cluster.protocol.message.DisconnectMessage;
+import org.apache.nifi.cluster.protocol.message.NodeStatusChangeMessage;
+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.events.EventReporter;
+import org.apache.nifi.reporting.Severity;
+import org.apache.nifi.services.FlowService;
+import org.apache.nifi.web.revision.RevisionManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+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.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandler, RequestCompletionCallback {
+ private static final Logger logger = LoggerFactory.getLogger(NodeClusterCoordinator.class);
+ private static final String EVENT_CATEGORY = "Clustering";
+
+ private static final Pattern COUNTER_URI_PATTERN = Pattern.compile("/nifi-api/controller/counters/[a-f0-9\\-]{36}");
+
+ private final String instanceId = UUID.randomUUID().toString();
+ private volatile NodeIdentifier nodeId;
+
+ private final ClusterCoordinationProtocolSenderListener senderListener;
+ private final EventReporter eventReporter;
+ private final ClusterNodeFirewall firewall;
+ private final RevisionManager revisionManager;
+ private volatile FlowService flowService;
+ private volatile boolean connected;
+
+ 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) {
+ this.senderListener = senderListener;
+ this.flowService = null;
+ this.eventReporter = eventReporter;
+ this.firewall = firewall;
+ this.revisionManager = revisionManager;
+ senderListener.addHandler(this);
+ }
+
+ @Override
+ public void shutdown() {
+ final NodeConnectionStatus shutdownStatus = new NodeConnectionStatus(getLocalNodeIdentifier(), DisconnectionCode.NODE_SHUTDOWN);
+ notifyOthersOfNodeStatusChange(shutdownStatus);
+ logger.info("Successfully notified other nodes that I am shutting down");
+ }
+
+ @Override
+ public void setLocalNodeIdentifier(final NodeIdentifier nodeId) {
+ this.nodeId = nodeId;
+ }
+
+ NodeIdentifier getLocalNodeIdentifier() {
+ return nodeId;
+ }
+
+ @Override
+ public void resetNodeStatuses(final Map statusMap) {
+ logger.info("Resetting cluster node statuses from {} to {}", nodeStatuses, statusMap);
+
+ // For each proposed replacement, update the nodeStatuses map if and only if the replacement
+ // has a larger update id than the current value.
+ for (final Map.Entry entry : statusMap.entrySet()) {
+ final NodeIdentifier nodeId = entry.getKey();
+ final NodeConnectionStatus proposedStatus = entry.getValue();
+
+ boolean updated = false;
+ while (!updated) {
+ final NodeConnectionStatus currentStatus = nodeStatuses.get(nodeId);
+
+ if (currentStatus == null || proposedStatus.getUpdateIdentifier() > currentStatus.getUpdateIdentifier()) {
+ updated = replaceNodeStatus(nodeId, currentStatus, proposedStatus);
+ } else {
+ updated = true;
+ }
+ }
+ }
+ }
+
+ /**
+ * Attempts to update the nodeStatuses map by changing the value for the given node id from the current status to the new status, as in
+ * ConcurrentMap.replace(nodeId, currentStatus, newStatus) but with the difference that this method can handle a null
value
+ * for currentStatus
+ *
+ * @param nodeId the node id
+ * @param currentStatus the current status, or null
if there is no value currently
+ * @param newStatus the new status to set
+ * @return true
if the map was updated, false otherwise
+ */
+ private boolean replaceNodeStatus(final NodeIdentifier nodeId, final NodeConnectionStatus currentStatus, final NodeConnectionStatus newStatus) {
+ if (newStatus == null) {
+ logger.error("Cannot change node status for {} from {} to {} because new status is null", nodeId, currentStatus, newStatus);
+ logger.error("", new NullPointerException());
+ }
+
+ if (currentStatus == null) {
+ final NodeConnectionStatus existingValue = nodeStatuses.putIfAbsent(nodeId, newStatus);
+ return existingValue == null;
+ }
+
+ return nodeStatuses.replace(nodeId, currentStatus, newStatus);
+ }
+
+ @Override
+ public void requestNodeConnect(final NodeIdentifier nodeId, final String userDn) {
+ if (userDn == null) {
+ reportEvent(nodeId, Severity.INFO, "Requesting that node connect to cluster");
+ } else {
+ reportEvent(nodeId, Severity.INFO, "Requesting that node connect to cluster on behalf of " + userDn);
+ }
+
+ updateNodeStatus(new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTING, System.currentTimeMillis()));
+
+ // create the request
+ final ReconnectionRequestMessage request = new ReconnectionRequestMessage();
+ request.setNodeId(nodeId);
+ request.setInstanceId(instanceId);
+
+ requestReconnectionAsynchronously(request, 10, 5);
+ }
+
+ @Override
+ public void finishNodeConnection(final NodeIdentifier nodeId) {
+ final NodeConnectionState state = getConnectionState(nodeId);
+ if (state == null) {
+ logger.debug("Attempted to finish node connection for {} but node is not known. Requesting that node connect", nodeId);
+ requestNodeConnect(nodeId, null);
+ return;
+ }
+
+ if (state == NodeConnectionState.CONNECTED) {
+ // already connected. Nothing to do.
+ return;
+ }
+
+ if (state == NodeConnectionState.DISCONNECTED || state == NodeConnectionState.DISCONNECTING) {
+ logger.debug("Attempted to finish node connection for {} but node state was {}. Requesting that node connect", nodeId, state);
+ requestNodeConnect(nodeId, null);
+ return;
+ }
+
+ logger.info("{} is now connected", nodeId);
+ final boolean updated = updateNodeStatus(new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTED));
+ if (!updated) {
+ logger.error("Attempting to Finish Node Connection but could not find Node with Identifier {}", nodeId);
+ }
+ }
+
+ @Override
+ public void requestNodeDisconnect(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode, final String explanation) {
+ logger.info("Requesting that {} disconnect due to {}", nodeId, explanation == null ? disconnectionCode : explanation);
+
+ updateNodeStatus(new NodeConnectionStatus(nodeId, disconnectionCode, explanation));
+
+ // There is no need to tell the node that it's disconnected if it is due to being
+ // shutdown, as we will not be able to connect to the node anyway.
+ if (disconnectionCode == DisconnectionCode.NODE_SHUTDOWN) {
+ return;
+ }
+
+ final DisconnectMessage request = new DisconnectMessage();
+ request.setNodeId(nodeId);
+ request.setExplanation(explanation);
+
+ addNodeEvent(nodeId, "Disconnection requested due to " + explanation);
+ disconnectAsynchronously(request, 10, 5);
+ }
+
+ @Override
+ public void disconnectionRequestedByNode(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode, final String explanation) {
+ logger.info("{} requested disconnection from cluster due to {}", nodeId, explanation == null ? disconnectionCode : explanation);
+ updateNodeStatus(new NodeConnectionStatus(nodeId, disconnectionCode, explanation));
+
+ final Severity severity;
+ switch (disconnectionCode) {
+ case STARTUP_FAILURE:
+ case MISMATCHED_FLOWS:
+ case UNKNOWN:
+ severity = Severity.ERROR;
+ break;
+ default:
+ severity = Severity.INFO;
+ break;
+ }
+
+ reportEvent(nodeId, severity, "Node disconnected from cluster due to " + explanation);
+ }
+
+ @Override
+ 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));
+ }
+
+ @Override
+ public NodeConnectionStatus getConnectionStatus(final NodeIdentifier nodeId) {
+ return nodeStatuses.get(nodeId);
+ }
+
+ private NodeConnectionState getConnectionState(final NodeIdentifier nodeId) {
+ final NodeConnectionStatus status = getConnectionStatus(nodeId);
+ return status == null ? null : status.getState();
+ }
+
+
+ @Override
+ public Map> getConnectionStates() {
+ final Map> connectionStates = new HashMap<>();
+ for (final Map.Entry entry : nodeStatuses.entrySet()) {
+ final NodeConnectionState state = entry.getValue().getState();
+ final List nodeIds = connectionStates.computeIfAbsent(state, s -> new ArrayList());
+ nodeIds.add(entry.getKey());
+ }
+
+ return connectionStates;
+ }
+
+ @Override
+ public boolean isBlockedByFirewall(final String hostname) {
+ return firewall != null && !firewall.isPermissible(hostname);
+ }
+
+ @Override
+ public void reportEvent(final NodeIdentifier nodeId, final Severity severity, final String event) {
+ eventReporter.reportEvent(severity, nodeId == null ? EVENT_CATEGORY : nodeId.toString(), event);
+ if (nodeId != null) {
+ addNodeEvent(nodeId, severity, event);
+ }
+
+ final String message = nodeId == null ? event : "Event Reported for " + nodeId + ": " + event;
+ switch (severity) {
+ case ERROR:
+ logger.error(message);
+ break;
+ case WARNING:
+ logger.warn(message);
+ break;
+ case INFO:
+ logger.info(message);
+ break;
+ }
+ }
+
+ @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 IllegalStateException("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) {
+ for (final NodeIdentifier nodeId : nodeStatuses.keySet()) {
+ if (nodeId.getId().equals(uuid)) {
+ return nodeId;
+ }
+ }
+
+ return null;
+ }
+
+
+ @Override
+ public Set getNodeIdentifiers(final NodeConnectionState... states) {
+ final Set