From 07d50c43a4eca996bee6f7673c59c803e3131b6b Mon Sep 17 00:00:00 2001 From: Jeff Storck Date: Tue, 18 Sep 2018 17:09:13 -0400 Subject: [PATCH 1/5] NIFI-5585 Added capability to offload a node that is disconnected from the cluster. Updated NodeClusterCoordinator to allow idempotent requests to offload a cluster Added capability to connect/delete/disconnect/offload a node from the cluster to the Toolkit CLI Added capability to get the status of nodes from the cluster to the Toolkit CLI Upgraded FontAwesome to 4.7.0 (from 4.6.1) Added icon "fa-upload" for offloading nodes in the cluster table UI --- .../nifi/controller/queue/FlowFileQueue.java | 2 + .../coordination/ClusterCoordinator.java | 18 ++++ .../ClusterTopologyEventListener.java | 2 + .../node/NodeConnectionState.java | 10 ++ .../node/NodeConnectionStatus.java | 45 +++++---- .../coordination/node/OffloadCode.java | 40 ++++++++ .../ClusterCoordinationProtocolSender.java | 9 ++ ...terCoordinationProtocolSenderListener.java | 6 ++ .../protocol/impl/SocketProtocolListener.java | 3 + ...dardClusterCoordinationProtocolSender.java | 26 +++++ .../message/AdaptedNodeConnectionStatus.java | 20 +++- .../message/NodeConnectionStatusAdapter.java | 6 +- .../protocol/jaxb/message/ObjectFactory.java | 5 + .../protocol/message/OffloadMessage.java | 53 +++++++++++ .../protocol/message/ProtocolMessage.java | 1 + .../heartbeat/AbstractHeartbeatMonitor.java | 8 +- .../ThreadPoolRequestReplicator.java | 18 ++++ .../node/NodeClusterCoordinator.java | 94 +++++++++++++++++-- .../IllegalNodeOffloadException.java | 38 ++++++++ .../OffloadedNodeMutableRequestException.java | 39 ++++++++ .../TestAbstractHeartbeatMonitor.java | 11 +++ .../node/TestNodeClusterCoordinator.java | 4 +- .../nifi/controller/StandardFlowService.java | 71 +++++++++++++- .../queue/StandardFlowFileQueue.java | 4 + .../SocketLoadBalancedFlowFileQueue.java | 34 +++++++ .../NonLocalPartitionPartitioner.java | 58 ++++++++++++ .../TestWriteAheadFlowFileRepository.java | 4 + .../nifi/web/StandardNiFiServiceFacade.java | 6 +- .../IllegalNodeOffloadExceptionMapper.java | 46 +++++++++ .../src/main/frontend/package.json | 2 +- .../webapp/js/nf/cluster/nf-cluster-table.js | 60 +++++++++++- .../impl/client/nifi/ControllerClient.java | 14 +++ .../nifi/impl/JerseyControllerClient.java | 87 +++++++++++++++++ .../cli/impl/command/CommandOption.java | 3 + .../impl/command/nifi/NiFiCommandGroup.java | 12 +++ .../impl/command/nifi/nodes/ConnectNode.java | 67 +++++++++++++ .../impl/command/nifi/nodes/DeleteNode.java | 58 ++++++++++++ .../command/nifi/nodes/DisconnectNode.java | 67 +++++++++++++ .../cli/impl/command/nifi/nodes/GetNode.java | 59 ++++++++++++ .../cli/impl/command/nifi/nodes/GetNodes.java | 52 ++++++++++ .../impl/command/nifi/nodes/OffloadNode.java | 67 +++++++++++++ .../toolkit/cli/impl/result/NodeResult.java | 48 ++++++++++ .../toolkit/cli/impl/result/NodesResult.java | 66 +++++++++++++ 43 files changed, 1299 insertions(+), 44 deletions(-) create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/OffloadCode.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/OffloadMessage.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeOffloadException.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/OffloadedNodeMutableRequestException.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/partition/NonLocalPartitionPartitioner.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/config/IllegalNodeOffloadExceptionMapper.java create mode 100644 nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/nodes/ConnectNode.java create mode 100644 nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/nodes/DeleteNode.java create mode 100644 nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/nodes/DisconnectNode.java create mode 100644 nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/nodes/GetNode.java create mode 100644 nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/nodes/GetNodes.java create mode 100644 nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/nodes/OffloadNode.java create mode 100644 nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/result/NodeResult.java create mode 100644 nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/result/NodesResult.java diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/queue/FlowFileQueue.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/queue/FlowFileQueue.java index 2c7f55b5ae4b..7cd0e3039347 100644 --- a/nifi-framework-api/src/main/java/org/apache/nifi/controller/queue/FlowFileQueue.java +++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/queue/FlowFileQueue.java @@ -267,6 +267,8 @@ public interface FlowFileQueue { void setLoadBalanceStrategy(LoadBalanceStrategy strategy, String partitioningAttribute); + void offloadQueue(); + LoadBalanceStrategy getLoadBalanceStrategy(); void setLoadBalanceCompression(LoadBalanceCompression compression); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/ClusterCoordinator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/ClusterCoordinator.java index 11786c27c037..2ad0e709718b 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/ClusterCoordinator.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/ClusterCoordinator.java @@ -17,6 +17,7 @@ package org.apache.nifi.cluster.coordination; +import org.apache.nifi.cluster.coordination.node.OffloadCode; import org.apache.nifi.cluster.coordination.node.DisconnectionCode; import org.apache.nifi.cluster.coordination.node.NodeConnectionState; import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; @@ -61,6 +62,23 @@ public interface ClusterCoordinator { */ void finishNodeConnection(NodeIdentifier nodeId); + /** + * Indicates that the node has finished being offloaded + * + * @param nodeId the identifier of the node + */ + void finishNodeOffload(NodeIdentifier nodeId); + + /** + * Sends a request to the node to be offloaded. + * The node will be marked as offloading immediately. + * + * @param nodeId the identifier of the node + * @param offloadCode the code that represents why this node is being asked to be offloaded + * @param explanation an explanation as to why the node is being asked to be offloaded + */ + void requestNodeOffload(NodeIdentifier nodeId, OffloadCode offloadCode, String explanation); + /** * Sends a request to the node to disconnect from the cluster. * The node will be marked as disconnected immediately. diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/ClusterTopologyEventListener.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/ClusterTopologyEventListener.java index 54cc4de1179d..ad9be3d738e5 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/ClusterTopologyEventListener.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/ClusterTopologyEventListener.java @@ -23,6 +23,8 @@ public interface ClusterTopologyEventListener { void onNodeAdded(NodeIdentifier nodeId); + void onNodeOffloaded(NodeIdentifier nodeId); + void onNodeRemoved(NodeIdentifier nodeId); void onLocalNodeIdentifierSet(NodeIdentifier localNodeId); 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 8d5824f17103..d79552c8cdb8 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 @@ -36,12 +36,22 @@ public enum NodeConnectionState { */ CONNECTED, + /** + * A node that is in the process of offloading its flow files from the node. + */ + OFFLOADING, + /** * A node that is in the process of disconnecting from the cluster. * A DISCONNECTING node will always transition to DISCONNECTED. */ DISCONNECTING, + /** + * A node that has offloaded its flow files from the node. + */ + OFFLOADED, + /** * A node that is not connected to the cluster. * A DISCONNECTED node can transition to CONNECTING. 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 34bd1279e3e7..7d8a94049cc5 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 @@ -35,47 +35,53 @@ public class NodeConnectionStatus { private final long updateId; private final NodeIdentifier nodeId; private final NodeConnectionState state; + private final OffloadCode offloadCode; private final DisconnectionCode disconnectCode; - private final String disconnectReason; + private final String reason; private final Long connectionRequestTime; public NodeConnectionStatus(final NodeIdentifier nodeId, final NodeConnectionState state) { - this(nodeId, state, null, null, null); + this(nodeId, state, null, null, null, null); } public NodeConnectionStatus(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode) { - this(nodeId, NodeConnectionState.DISCONNECTED, disconnectionCode, disconnectionCode.toString(), null); + this(nodeId, NodeConnectionState.DISCONNECTED, null, disconnectionCode, disconnectionCode.toString(), null); + } + + public NodeConnectionStatus(final NodeIdentifier nodeId, final NodeConnectionState state, final OffloadCode offloadCode, final String offloadExplanation) { + this(nodeId, state, offloadCode, null, offloadExplanation, null); } public NodeConnectionStatus(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode, final String disconnectionExplanation) { - this(nodeId, NodeConnectionState.DISCONNECTED, disconnectionCode, disconnectionExplanation, null); + this(nodeId, NodeConnectionState.DISCONNECTED, null, disconnectionCode, disconnectionExplanation, null); } public NodeConnectionStatus(final NodeIdentifier nodeId, final NodeConnectionState state, final DisconnectionCode disconnectionCode) { - this(nodeId, state, disconnectionCode, disconnectionCode == null ? null : disconnectionCode.toString(), null); + this(nodeId, state, null, disconnectionCode, disconnectionCode == null ? null : disconnectionCode.toString(), null); } public NodeConnectionStatus(final NodeConnectionStatus status) { - this(status.getNodeIdentifier(), status.getState(), status.getDisconnectCode(), status.getDisconnectReason(), status.getConnectionRequestTime()); + this(status.getNodeIdentifier(), status.getState(), status.getOffloadCode(), status.getDisconnectCode(), status.getReason(), status.getConnectionRequestTime()); } - public NodeConnectionStatus(final NodeIdentifier nodeId, final NodeConnectionState state, final DisconnectionCode disconnectCode, - final String disconnectReason, final Long connectionRequestTime) { - this(idGenerator.getAndIncrement(), nodeId, state, disconnectCode, disconnectReason, connectionRequestTime); + public NodeConnectionStatus(final NodeIdentifier nodeId, final NodeConnectionState state, final OffloadCode offloadCode, + final DisconnectionCode disconnectCode, final String reason, final Long connectionRequestTime) { + this(idGenerator.getAndIncrement(), nodeId, state, offloadCode, disconnectCode, reason, connectionRequestTime); } - public NodeConnectionStatus(final long updateId, final NodeIdentifier nodeId, final NodeConnectionState state, final DisconnectionCode disconnectCode, - final String disconnectReason, final Long connectionRequestTime) { + public NodeConnectionStatus(final long updateId, final NodeIdentifier nodeId, final NodeConnectionState state, final OffloadCode offloadCode, + final DisconnectionCode disconnectCode, final String reason, final Long connectionRequestTime) { this.updateId = updateId; this.nodeId = nodeId; this.state = state; + this.offloadCode = offloadCode; if (state == NodeConnectionState.DISCONNECTED && disconnectCode == null) { this.disconnectCode = DisconnectionCode.UNKNOWN; - this.disconnectReason = this.disconnectCode.toString(); + this.reason = this.disconnectCode.toString(); } else { this.disconnectCode = disconnectCode; - this.disconnectReason = disconnectReason; + this.reason = reason; } this.connectionRequestTime = (connectionRequestTime == null && state == NodeConnectionState.CONNECTING) ? Long.valueOf(System.currentTimeMillis()) : connectionRequestTime; @@ -93,12 +99,16 @@ public NodeConnectionState getState() { return state; } + public OffloadCode getOffloadCode() { + return offloadCode; + } + public DisconnectionCode getDisconnectCode() { return disconnectCode; } - public String getDisconnectReason() { - return disconnectReason; + public String getReason() { + return reason; } public Long getConnectionRequestTime() { @@ -110,8 +120,11 @@ public String toString() { final StringBuilder sb = new StringBuilder(); final NodeConnectionState state = getState(); sb.append("NodeConnectionStatus[nodeId=").append(nodeId).append(", state=").append(state); + if (state == NodeConnectionState.OFFLOADED || state == NodeConnectionState.OFFLOADING) { + sb.append(", Offload Code=").append(getOffloadCode()).append(", Offload Reason=").append(getReason()); + } if (state == NodeConnectionState.DISCONNECTED || state == NodeConnectionState.DISCONNECTING) { - sb.append(", Disconnect Code=").append(getDisconnectCode()).append(", Disconnect Reason=").append(getDisconnectReason()); + sb.append(", Disconnect Code=").append(getDisconnectCode()).append(", Disconnect Reason=").append(getReason()); } sb.append(", updateId=").append(getUpdateIdentifier()); sb.append("]"); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/OffloadCode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/OffloadCode.java new file mode 100644 index 000000000000..fb4d30bbc5f0 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/OffloadCode.java @@ -0,0 +1,40 @@ +/* + * 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; + +/** + * An enumeration of the reasons that a node may be offloaded + */ +public enum OffloadCode { + + /** + * A user explicitly offloaded the node + */ + OFFLOADED("Node Offloaded"); + + private final String description; + + OffloadCode(final String description) { + this.description = description; + } + + @Override + public String toString() { + return description; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterCoordinationProtocolSender.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterCoordinationProtocolSender.java index 986231efd466..b5485ccd5652 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterCoordinationProtocolSender.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,6 +19,7 @@ import java.util.Set; import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; +import org.apache.nifi.cluster.protocol.message.OffloadMessage; import org.apache.nifi.cluster.protocol.message.DisconnectMessage; import org.apache.nifi.cluster.protocol.message.NodeStatusChangeMessage; import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage; @@ -40,6 +41,14 @@ public interface ClusterCoordinationProtocolSender { */ ReconnectionResponseMessage requestReconnection(ReconnectionRequestMessage msg) throws ProtocolException; + /** + * Sends an "offload request" message to a node. + * + * @param msg a message + * @throws ProtocolException if communication failed + */ + void offload(OffloadMessage msg) throws ProtocolException; + /** * Sends a "disconnection 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/impl/ClusterCoordinationProtocolSenderListener.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 index ae3a0e50571e..74cc6b476a25 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterCoordinationProtocolSenderListener.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 @@ -26,6 +26,7 @@ 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.OffloadMessage; import org.apache.nifi.cluster.protocol.message.DisconnectMessage; import org.apache.nifi.cluster.protocol.message.NodeStatusChangeMessage; import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage; @@ -100,6 +101,11 @@ public ReconnectionResponseMessage requestReconnection(final ReconnectionRequest return sender.requestReconnection(msg); } + @Override + public void offload(OffloadMessage msg) throws ProtocolException { + sender.offload(msg); + } + @Override public void disconnect(DisconnectMessage msg) throws ProtocolException { sender.disconnect(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/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 9eaffd37b96a..c588a6807dc2 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 @@ -24,6 +24,7 @@ import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller; import org.apache.nifi.cluster.protocol.ProtocolMessageUnmarshaller; import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage; +import org.apache.nifi.cluster.protocol.message.OffloadMessage; import org.apache.nifi.cluster.protocol.message.DisconnectMessage; import org.apache.nifi.cluster.protocol.message.FlowRequestMessage; import org.apache.nifi.cluster.protocol.message.HeartbeatMessage; @@ -210,6 +211,8 @@ private NodeIdentifier getNodeIdentifier(final ProtocolMessage message) { return ((ConnectionRequestMessage) message).getConnectionRequest().getProposedNodeIdentifier(); case HEARTBEAT: return ((HeartbeatMessage) message).getHeartbeat().getNodeIdentifier(); + case OFFLOAD_REQUEST: + return ((OffloadMessage) message).getNodeId(); case DISCONNECTION_REQUEST: return ((DisconnectMessage) message).getNodeId(); case FLOW_REQUEST: diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/StandardClusterCoordinationProtocolSender.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 index 167ddec93284..b21068ffe5e2 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/StandardClusterCoordinationProtocolSender.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 @@ -36,6 +36,7 @@ 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.OffloadMessage; import org.apache.nifi.cluster.protocol.message.DisconnectMessage; import org.apache.nifi.cluster.protocol.message.NodeConnectionStatusRequestMessage; import org.apache.nifi.cluster.protocol.message.NodeConnectionStatusResponseMessage; @@ -128,6 +129,31 @@ public ReconnectionResponseMessage requestReconnection(final ReconnectionRequest } } + /** + * Requests a node to be offloaded. The configured value for + * handshake timeout is applied to the socket before making the request. + * + * @param msg a message + * @throws ProtocolException if the message failed to be sent + */ + @Override + public void offload(final OffloadMessage msg) throws ProtocolException { + Socket socket = null; + try { + socket = createSocket(msg.getNodeId(), true); + + // marshal message to output stream + try { + 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); + } + } finally { + SocketUtils.closeQuietly(socket); + } + } + /** * Requests a node to disconnect from the cluster. The configured value for * handshake timeout is applied to the socket before making the request. 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 c8c4acf646a3..5eae83e0e11b 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,6 +17,7 @@ package org.apache.nifi.cluster.protocol.jaxb.message; +import org.apache.nifi.cluster.coordination.node.OffloadCode; import org.apache.nifi.cluster.coordination.node.DisconnectionCode; import org.apache.nifi.cluster.coordination.node.NodeConnectionState; import org.apache.nifi.cluster.protocol.NodeIdentifier; @@ -25,8 +26,9 @@ public class AdaptedNodeConnectionStatus { private Long updateId; private NodeIdentifier nodeId; private NodeConnectionState state; + private OffloadCode offloadCode; private DisconnectionCode disconnectCode; - private String disconnectReason; + private String reason; private Long connectionRequestTime; public Long getUpdateId() { @@ -53,20 +55,28 @@ public void setState(NodeConnectionState state) { this.state = state; } + public OffloadCode getOffloadCode() { + return offloadCode; + } + public DisconnectionCode getDisconnectCode() { return disconnectCode; } + public void setOffloadCode(OffloadCode offloadCode) { + this.offloadCode = offloadCode; + } + public void setDisconnectCode(DisconnectionCode disconnectCode) { this.disconnectCode = disconnectCode; } - public String getDisconnectReason() { - return disconnectReason; + public String getReason() { + return reason; } - public void setDisconnectReason(String disconnectReason) { - this.disconnectReason = disconnectReason; + public void setReason(String reason) { + this.reason = reason; } public Long getConnectionRequestTime() { 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 ec209de1f540..47e92e8d2a34 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 @@ -28,8 +28,9 @@ public NodeConnectionStatus unmarshal(final AdaptedNodeConnectionStatus adapted) return new NodeConnectionStatus(adapted.getUpdateId(), adapted.getNodeId(), adapted.getState(), + adapted.getOffloadCode(), adapted.getDisconnectCode(), - adapted.getDisconnectReason(), + adapted.getReason(), adapted.getConnectionRequestTime()); } @@ -40,8 +41,9 @@ public AdaptedNodeConnectionStatus marshal(final NodeConnectionStatus toAdapt) t adapted.setUpdateId(toAdapt.getUpdateIdentifier()); adapted.setNodeId(toAdapt.getNodeIdentifier()); adapted.setConnectionRequestTime(toAdapt.getConnectionRequestTime()); + adapted.setOffloadCode(toAdapt.getOffloadCode()); adapted.setDisconnectCode(toAdapt.getDisconnectCode()); - adapted.setDisconnectReason(toAdapt.getDisconnectReason()); + adapted.setReason(toAdapt.getReason()); adapted.setState(toAdapt.getState()); } return adapted; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ObjectFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ObjectFactory.java index 9a594a403e8d..2f02e5e6fd5b 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ObjectFactory.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ObjectFactory.java @@ -20,6 +20,7 @@ import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage; import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage; +import org.apache.nifi.cluster.protocol.message.OffloadMessage; import org.apache.nifi.cluster.protocol.message.DisconnectMessage; import org.apache.nifi.cluster.protocol.message.FlowRequestMessage; import org.apache.nifi.cluster.protocol.message.FlowResponseMessage; @@ -52,6 +53,10 @@ public ReconnectionResponseMessage createReconnectionResponseMessage() { return new ReconnectionResponseMessage(); } + public OffloadMessage createDecomissionMessage() { + return new OffloadMessage(); + } + public DisconnectMessage createDisconnectionMessage() { return new DisconnectMessage(); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/OffloadMessage.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/OffloadMessage.java new file mode 100644 index 000000000000..a7acd56ad60e --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/OffloadMessage.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.protocol.message; + +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.cluster.protocol.jaxb.message.NodeIdentifierAdapter; + +import javax.xml.bind.annotation.XmlRootElement; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; + +@XmlRootElement(name = "offloadMessage") +public class OffloadMessage extends ProtocolMessage { + + private NodeIdentifier nodeId; + private String explanation; + + @XmlJavaTypeAdapter(NodeIdentifierAdapter.class) + public NodeIdentifier getNodeId() { + return nodeId; + } + + public void setNodeId(NodeIdentifier nodeId) { + this.nodeId = nodeId; + } + + public String getExplanation() { + return explanation; + } + + public void setExplanation(String explanation) { + this.explanation = explanation; + } + + @Override + public MessageType getType() { + return MessageType.OFFLOAD_REQUEST; + } + +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ProtocolMessage.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ProtocolMessage.java index 482f5d6bc012..fe26c7a2cc12 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ProtocolMessage.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ProtocolMessage.java @@ -21,6 +21,7 @@ public abstract class ProtocolMessage { public static enum MessageType { CONNECTION_REQUEST, CONNECTION_RESPONSE, + OFFLOAD_REQUEST, DISCONNECTION_REQUEST, EXCEPTION, FLOW_REQUEST, 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 43958833c241..f6d09ab32e95 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 @@ -228,6 +228,12 @@ private void processHeartbeat(final NodeHeartbeat heartbeat) { return; } + if (NodeConnectionState.OFFLOADED == connectionState) { + // Cluster Coordinator believes that node is offloaded, but let the node reconnect + clusterCoordinator.reportEvent(nodeId, Severity.INFO, "Received heartbeat from node that is offloaded. " + + "Marking as Disconnected and requesting that Node reconnect to cluster"); + clusterCoordinator.requestNodeConnect(nodeId, null); + } 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 @@ -249,7 +255,7 @@ private void processHeartbeat(final NodeHeartbeat heartbeat) { default: { // 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, disconnectionCode, connectionStatus.getDisconnectReason()); + clusterCoordinator.requestNodeDisconnect(nodeId, disconnectionCode, connectionStatus.getReason()); removeHeartbeat(nodeId); break; } 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 85618deef709..0b2f1fe4ecea 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 @@ -170,6 +170,24 @@ public AsyncClusterResponse replicate(NiFiUser user, String method, URI uri, Obj // If the request is mutable, ensure that all nodes are connected. if (mutable) { + final List offloaded = stateMap.get(NodeConnectionState.OFFLOADED); + if (offloaded != null && !offloaded.isEmpty()) { + if (offloaded.size() == 1) { + throw new OffloadedNodeMutableRequestException("Node " + offloaded.iterator().next() + " is currently offloaded"); + } else { + throw new OffloadedNodeMutableRequestException(offloaded.size() + " Nodes are currently offloaded"); + } + } + + final List offloading = stateMap.get(NodeConnectionState.OFFLOADING); + if (offloading != null && !offloading.isEmpty()) { + if (offloading.size() == 1) { + throw new OffloadedNodeMutableRequestException("Node " + offloading.iterator().next() + " is currently offloading"); + } else { + throw new OffloadedNodeMutableRequestException(offloading.size() + " Nodes are currently offloading"); + } + } + final List disconnected = stateMap.get(NodeConnectionState.DISCONNECTED); if (disconnected != null && !disconnected.isEmpty()) { if (disconnected.size() == 1) { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java index 484d1556c11a..65a0a390d843 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java @@ -34,6 +34,7 @@ import org.apache.nifi.cluster.exception.NoClusterCoordinatorException; import org.apache.nifi.cluster.firewall.ClusterNodeFirewall; import org.apache.nifi.cluster.manager.NodeResponse; +import org.apache.nifi.cluster.manager.exception.IllegalNodeOffloadException; import org.apache.nifi.cluster.manager.exception.IllegalNodeDisconnectionException; import org.apache.nifi.cluster.protocol.ComponentRevision; import org.apache.nifi.cluster.protocol.ConnectionRequest; @@ -49,6 +50,7 @@ import org.apache.nifi.cluster.protocol.message.ClusterWorkloadResponseMessage; import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage; import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage; +import org.apache.nifi.cluster.protocol.message.OffloadMessage; import org.apache.nifi.cluster.protocol.message.DisconnectMessage; import org.apache.nifi.cluster.protocol.message.NodeConnectionStatusResponseMessage; import org.apache.nifi.cluster.protocol.message.NodeStatusChangeMessage; @@ -431,7 +433,7 @@ public void requestNodeConnect(final NodeIdentifier nodeId, final String userDn) reportEvent(nodeId, Severity.INFO, "Requesting that node connect to cluster on behalf of " + userDn); } - updateNodeStatus(new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTING, null, null, System.currentTimeMillis())); + updateNodeStatus(new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTING, null, null, null, System.currentTimeMillis())); // create the request final ReconnectionRequestMessage request = new ReconnectionRequestMessage(); @@ -469,6 +471,50 @@ public void finishNodeConnection(final NodeIdentifier nodeId) { updateNodeStatus(new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTED)); } + @Override + public void finishNodeOffload(final NodeIdentifier nodeId) { + final NodeConnectionState state = getConnectionState(nodeId); + if (state == null) { + logger.warn("Attempted to finish node offload for {} but node is not known.", nodeId); + return; + } + + if (state != NodeConnectionState.OFFLOADING) { + logger.warn("Attempted to finish node offload for {} but node is not in a offload state, it is currently {}.", nodeId, state); + return; + } + + logger.info("{} is now offloaded", nodeId); + updateNodeStatus(new NodeConnectionStatus(nodeId, NodeConnectionState.OFFLOADED)); + } + + @Override + public void requestNodeOffload(final NodeIdentifier nodeId, final OffloadCode offloadCode, final String explanation) { + final Set offloadNodeIds = getNodeIdentifiers(NodeConnectionState.OFFLOADING, NodeConnectionState.OFFLOADED); + if (offloadNodeIds.contains(nodeId)) { + logger.debug("Attempted to offload node but the node is already offloading or offloaded"); + // no need to do anything here, the node is currently offloading or already offloaded + return; + } + + final Set disconnectedNodeIds = getNodeIdentifiers(NodeConnectionState.DISCONNECTED); + if (!disconnectedNodeIds.contains(nodeId)) { + throw new IllegalNodeOffloadException("Cannot offload node " + nodeId + " because it is not currently disconnected"); + } + + logger.info("Requesting that {} is offloaded due to {}", nodeId, explanation == null ? offloadCode : explanation); + + updateNodeStatus(new NodeConnectionStatus(nodeId, NodeConnectionState.OFFLOADING, offloadCode, explanation)); + + final OffloadMessage request = new OffloadMessage(); + request.setNodeId(nodeId); + request.setExplanation(explanation); + + addNodeEvent(nodeId, "Offload requested due to " + explanation); + onNodeOffloaded(nodeId); + offloadAsynchronously(request, 10, 5); + } + @Override public void requestNodeDisconnect(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode, final String explanation) { final Set connectedNodeIds = getNodeIdentifiers(NodeConnectionState.CONNECTED); @@ -526,17 +572,19 @@ public void removeNode(final NodeIdentifier nodeId, final String userDn) { storeState(); } + private void onNodeOffloaded(final NodeIdentifier nodeId) { + eventListeners.forEach(listener -> listener.onNodeOffloaded(nodeId)); + } + private void onNodeRemoved(final NodeIdentifier nodeId) { - eventListeners.stream().forEach(listener -> listener.onNodeRemoved(nodeId)); + eventListeners.forEach(listener -> listener.onNodeRemoved(nodeId)); } private void onNodeAdded(final NodeIdentifier nodeId, final boolean storeState) { if (storeState) { storeState(); } - - - eventListeners.stream().forEach(listener -> listener.onNodeAdded(nodeId)); + eventListeners.forEach(listener -> listener.onNodeAdded(nodeId)); } @Override @@ -821,7 +869,7 @@ void notifyOthersOfNodeStatusChange(final NodeConnectionStatus updatedStatus, fi // Otherwise, get the active coordinator (or wait for one to become active) and then notify the coordinator. final Set nodesToNotify; if (notifyAllNodes) { - nodesToNotify = getNodeIdentifiers(NodeConnectionState.CONNECTED, NodeConnectionState.CONNECTING); + nodesToNotify = getNodeIdentifiers(); // Do not notify ourselves because we already know about the status update. nodesToNotify.remove(getLocalNodeIdentifier()); @@ -841,6 +889,34 @@ void notifyOthersOfNodeStatusChange(final NodeConnectionStatus updatedStatus, fi senderListener.notifyNodeStatusChange(nodesToNotify, message); } + private void offloadAsynchronously(final OffloadMessage request, final int attempts, final int retrySeconds) { + final Thread offloadThread = new Thread(new Runnable() { + @Override + public void run() { + final NodeIdentifier nodeId = request.getNodeId(); + + for (int i = 0; i < attempts; i++) { + try { + senderListener.offload(request); + reportEvent(nodeId, Severity.INFO, "Node was offloaded due to " + request.getExplanation()); + return; + } catch (final Exception e) { + logger.error("Failed to notify {} that it has been offloaded due to {}", request.getNodeId(), request.getExplanation(), e); + + try { + Thread.sleep(retrySeconds * 1000L); + } catch (final InterruptedException ie) { + Thread.currentThread().interrupt(); + return; + } + } + } + } + }, "Offload " + request.getNodeId()); + + offloadThread.start(); + } + private void disconnectAsynchronously(final DisconnectMessage request, final int attempts, final int retrySeconds) { final Thread disconnectThread = new Thread(new Runnable() { @Override @@ -961,8 +1037,8 @@ private String summarizeStatusChange(final NodeConnectionStatus oldStatus, final if (oldStatus == null || status.getState() != oldStatus.getState()) { sb.append("Node Status changed from ").append(oldStatus == null ? "[Unknown Node]" : oldStatus.getState().toString()).append(" to ").append(status.getState().toString()); - if (status.getDisconnectReason() != null) { - sb.append(" due to ").append(status.getDisconnectReason()); + if (status.getReason() != null) { + sb.append(" due to ").append(status.getReason()); } else if (status.getDisconnectCode() != null) { sb.append(" due to ").append(status.getDisconnectCode().toString()); } @@ -1117,7 +1193,7 @@ private ConnectionResponseMessage createConnectionResponse(final ConnectionReque addNodeEvent(resolvedNodeIdentifier, "Connection requested from existing node. Setting status to connecting."); } - status = new NodeConnectionStatus(resolvedNodeIdentifier, NodeConnectionState.CONNECTING, null, null, System.currentTimeMillis()); + status = new NodeConnectionStatus(resolvedNodeIdentifier, NodeConnectionState.CONNECTING, null, null, null, System.currentTimeMillis()); updateNodeStatus(status); final ConnectionResponse response = new ConnectionResponse(resolvedNodeIdentifier, clusterDataFlow, instanceId, getConnectionStatuses(), diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeOffloadException.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeOffloadException.java new file mode 100644 index 000000000000..f1bc6694d0e5 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeOffloadException.java @@ -0,0 +1,38 @@ +/* + * 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.manager.exception; + +/** + * Represents the exceptional case when an offload request is issued to a node that cannot be offloaded (e.g., not currently disconnected). + */ +public class IllegalNodeOffloadException extends IllegalClusterStateException { + + public IllegalNodeOffloadException() { + } + + public IllegalNodeOffloadException(String msg) { + super(msg); + } + + public IllegalNodeOffloadException(Throwable cause) { + super(cause); + } + + public IllegalNodeOffloadException(String msg, Throwable cause) { + super(msg, cause); + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/OffloadedNodeMutableRequestException.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/OffloadedNodeMutableRequestException.java new file mode 100644 index 000000000000..36633497357a --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/OffloadedNodeMutableRequestException.java @@ -0,0 +1,39 @@ +/* + * 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.manager.exception; + +/** + * Represents the exceptional case when a HTTP request that may change a node's dataflow is to be replicated while one or more nodes are offloaded. + * + */ +public class OffloadedNodeMutableRequestException extends MutableRequestException { + + public OffloadedNodeMutableRequestException() { + } + + public OffloadedNodeMutableRequestException(String msg) { + super(msg); + } + + public OffloadedNodeMutableRequestException(Throwable cause) { + super(cause); + } + + public OffloadedNodeMutableRequestException(String msg, Throwable cause) { + super(msg, cause); + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java index 6ea019d9476a..4aeff7b3eb95 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java @@ -20,6 +20,7 @@ import org.apache.nifi.cluster.ReportedEvent; import org.apache.nifi.cluster.coordination.ClusterCoordinator; import org.apache.nifi.cluster.coordination.ClusterTopologyEventListener; +import org.apache.nifi.cluster.coordination.node.OffloadCode; import org.apache.nifi.cluster.coordination.node.DisconnectionCode; import org.apache.nifi.cluster.coordination.node.NodeConnectionState; import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; @@ -244,6 +245,16 @@ public synchronized void finishNodeConnection(NodeIdentifier nodeId) { statuses.put(nodeId, new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTED)); } + @Override + public synchronized void finishNodeOffload(NodeIdentifier nodeId) { + statuses.put(nodeId, new NodeConnectionStatus(nodeId, NodeConnectionState.OFFLOADED)); + } + + @Override + public synchronized void requestNodeOffload(NodeIdentifier nodeId, OffloadCode offloadCode, String explanation) { + statuses.put(nodeId, new NodeConnectionStatus(nodeId, NodeConnectionState.OFFLOADED)); + } + @Override public synchronized void requestNodeDisconnect(NodeIdentifier nodeId, DisconnectionCode disconnectionCode, String explanation) { statuses.put(nodeId, new NodeConnectionStatus(nodeId, NodeConnectionState.DISCONNECTED)); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/node/TestNodeClusterCoordinator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/node/TestNodeClusterCoordinator.java index fb06a15dec79..5ce2985c03d7 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/node/TestNodeClusterCoordinator.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/node/TestNodeClusterCoordinator.java @@ -280,7 +280,7 @@ public void testStatusChangesReplicated() throws InterruptedException, IOExcepti assertNotNull(statusChange); assertEquals(createNodeId(1), statusChange.getNodeIdentifier()); assertEquals(DisconnectionCode.NODE_SHUTDOWN, statusChange.getDisconnectCode()); - assertEquals("Unit Test", statusChange.getDisconnectReason()); + assertEquals("Unit Test", statusChange.getReason()); } @Test @@ -407,7 +407,7 @@ public void testUpdateNodeStatusOutOfOrder() throws InterruptedException { nodeStatuses.clear(); final NodeConnectionStatus oldStatus = new NodeConnectionStatus(-1L, nodeId1, NodeConnectionState.DISCONNECTED, - DisconnectionCode.BLOCKED_BY_FIREWALL, null, 0L); + null, DisconnectionCode.BLOCKED_BY_FIREWALL, null, 0L); final NodeStatusChangeMessage msg = new NodeStatusChangeMessage(); msg.setNodeId(nodeId1); msg.setNodeConnectionStatus(oldStatus); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java index 7a5c45e087df..297595f4dfee 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java @@ -23,6 +23,7 @@ import org.apache.nifi.bundle.Bundle; import org.apache.nifi.cluster.ConnectionException; import org.apache.nifi.cluster.coordination.ClusterCoordinator; +import org.apache.nifi.cluster.coordination.node.OffloadCode; import org.apache.nifi.cluster.coordination.node.DisconnectionCode; import org.apache.nifi.cluster.coordination.node.NodeConnectionState; import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; @@ -36,6 +37,7 @@ import org.apache.nifi.cluster.protocol.StandardDataFlow; import org.apache.nifi.cluster.protocol.impl.NodeProtocolSenderListener; import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage; +import org.apache.nifi.cluster.protocol.message.OffloadMessage; import org.apache.nifi.cluster.protocol.message.DisconnectMessage; import org.apache.nifi.cluster.protocol.message.FlowRequestMessage; import org.apache.nifi.cluster.protocol.message.FlowResponseMessage; @@ -44,12 +46,14 @@ import org.apache.nifi.cluster.protocol.message.ReconnectionResponseMessage; import org.apache.nifi.components.state.Scope; import org.apache.nifi.components.state.StateManager; +import org.apache.nifi.controller.queue.FlowFileQueue; import org.apache.nifi.controller.serialization.FlowSerializationException; import org.apache.nifi.controller.serialization.FlowSynchronizationException; import org.apache.nifi.encrypt.StringEncryptor; import org.apache.nifi.engine.FlowEngine; import org.apache.nifi.events.BulletinFactory; import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.groups.RemoteProcessGroup; import org.apache.nifi.lifecycle.LifeCycleStartException; import org.apache.nifi.logging.LogLevel; import org.apache.nifi.nar.NarClassLoaders; @@ -381,6 +385,7 @@ public void run() { public boolean canHandle(final ProtocolMessage msg) { switch (msg.getType()) { case RECONNECTION_REQUEST: + case OFFLOAD_REQUEST: case DISCONNECTION_REQUEST: case FLOW_REQUEST: return true; @@ -415,6 +420,22 @@ public void run() { return new ReconnectionResponseMessage(); } + case OFFLOAD_REQUEST: { + final Thread t = new Thread(new Runnable() { + @Override + public void run() { + try { + handleOffloadRequest((OffloadMessage) request); + } catch (InterruptedException e) { + throw new ProtocolException("Could not complete offload request", e); + } + } + }, "Offload Flow Files from Node"); + t.setDaemon(true); + t.start(); + + return null; + } case DISCONNECTION_REQUEST: { final Thread t = new Thread(new Runnable() { @Override @@ -561,7 +582,7 @@ private void handleConnectionFailure(final Exception ex) { private FlowResponseMessage handleFlowRequest(final FlowRequestMessage request) throws ProtocolException { readLock.lock(); try { - logger.info("Received flow request message from manager."); + logger.info("Received flow request message from cluster coordinator."); // create the response final FlowResponseMessage response = new FlowResponseMessage(); @@ -631,7 +652,7 @@ private NodeIdentifier getNodeId() { private void handleReconnectionRequest(final ReconnectionRequestMessage request) { try { - logger.info("Processing reconnection request from manager."); + logger.info("Processing reconnection request from cluster coordinator."); // reconnect ConnectionResponse connectionResponse = new ConnectionResponse(getNodeId(), request.getDataFlow(), @@ -662,8 +683,48 @@ private void handleReconnectionRequest(final ReconnectionRequestMessage request) } } + private void handleOffloadRequest(final OffloadMessage request) throws InterruptedException { + logger.info("Received offload request message from cluster coordinator with explanation: " + request.getExplanation()); + offload(request.getExplanation()); + } + + private void offload(final String explanation) throws InterruptedException { + writeLock.lock(); + try { + + logger.info("Offloading node due to " + explanation); + + // mark node as offloading + controller.setConnectionStatus(new NodeConnectionStatus(nodeId, NodeConnectionState.OFFLOADING, OffloadCode.OFFLOADED, explanation)); + // request to stop all processors on node + controller.stopAllProcessors(); + // request to stop all remote process groups + controller.getRootGroup().findAllRemoteProcessGroups().forEach(RemoteProcessGroup::stopTransmitting); + // terminate all processors + controller.getRootGroup().findAllProcessors() + // filter stream, only stopped processors can be terminated + .stream().filter(pn -> pn.getScheduledState() == ScheduledState.STOPPED) + .forEach(pn -> pn.getProcessGroup().terminateProcessor(pn)); + // offload all queues on node + controller.getAllQueues().forEach(FlowFileQueue::offloadQueue); + // wait for rebalance of flowfiles on all queues + while (controller.getControllerStatus().getQueuedCount() > 0) { + logger.debug("Offloading queues on node {}, remaining queued count: {}", getNodeId(), controller.getControllerStatus().getQueuedCount()); + Thread.sleep(1000); + } + // finish offload + controller.setConnectionStatus(new NodeConnectionStatus(nodeId, NodeConnectionState.OFFLOADED, OffloadCode.OFFLOADED, explanation)); + clusterCoordinator.finishNodeOffload(getNodeId()); + + logger.info("Node offloaded due to " + explanation); + + } finally { + writeLock.unlock(); + } + } + private void handleDisconnectionRequest(final DisconnectMessage request) { - logger.info("Received disconnection request message from manager with explanation: " + request.getExplanation()); + logger.info("Received disconnection request message from cluster coordinator with explanation: " + request.getExplanation()); disconnect(request.getExplanation()); } @@ -829,11 +890,11 @@ private ConnectionResponse connect(final boolean retryOnCommsFailure, final bool } } else if (response.getRejectionReason() != null) { logger.warn("Connection request was blocked by cluster coordinator with the explanation: " + response.getRejectionReason()); - // set response to null and treat a firewall blockage the same as getting no response from manager + // set response to null and treat a firewall blockage the same as getting no response from cluster coordinator response = null; break; } else { - // we received a successful connection response from manager + // we received a successful connection response from cluster coordinator break; } } catch (final NoClusterCoordinatorException ncce) { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/StandardFlowFileQueue.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/StandardFlowFileQueue.java index cab41e8a8eb6..ee222f4f2d89 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/StandardFlowFileQueue.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/StandardFlowFileQueue.java @@ -74,6 +74,10 @@ public void startLoadBalancing() { public void stopLoadBalancing() { } + @Override + public void offloadQueue() { + } + @Override public boolean isActivelyLoadBalancing() { return false; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/SocketLoadBalancedFlowFileQueue.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/SocketLoadBalancedFlowFileQueue.java index f2502000d671..3143332893e9 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/SocketLoadBalancedFlowFileQueue.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/SocketLoadBalancedFlowFileQueue.java @@ -40,6 +40,7 @@ import org.apache.nifi.controller.queue.clustered.partition.FlowFilePartitioner; import org.apache.nifi.controller.queue.clustered.partition.LocalPartitionPartitioner; import org.apache.nifi.controller.queue.clustered.partition.LocalQueuePartition; +import org.apache.nifi.controller.queue.clustered.partition.NonLocalPartitionPartitioner; import org.apache.nifi.controller.queue.clustered.partition.QueuePartition; import org.apache.nifi.controller.queue.clustered.partition.RebalancingPartition; import org.apache.nifi.controller.queue.clustered.partition.RemoteQueuePartition; @@ -113,6 +114,7 @@ public class SocketLoadBalancedFlowFileQueue extends AbstractFlowFileQueue imple private QueuePartition[] queuePartitions; private FlowFilePartitioner partitioner; private boolean stopped = true; + private boolean offloaded = false; public SocketLoadBalancedFlowFileQueue(final String identifier, final ConnectionEventListener eventListener, final ProcessScheduler scheduler, final FlowFileRepository flowFileRepo, @@ -204,6 +206,19 @@ public synchronized void setLoadBalanceStrategy(final LoadBalanceStrategy strate setFlowFilePartitioner(partitioner); } + @Override + public void offloadQueue() { + if (clusterCoordinator == null) { + // Not clustered, so don't change partitions + return; + } + + offloaded = true; + + // TODO need to be able to reset the partitioner to the previous partitioner if this node is reconnected to the cluster + setFlowFilePartitioner(new NonLocalPartitionPartitioner()); + } + public synchronized void startLoadBalancing() { logger.debug("{} started. Will begin distributing FlowFiles across the cluster", this); @@ -551,6 +566,11 @@ public void setNodeIdentifiers(final Set updatedNodeIdentifiers, return; } + if (offloaded) { + logger.debug("{} Not going to rebalance Queue even though setNodeIdentifiers was called, because the queue has been offloaded", this); + return; + } + logger.debug("{} Stopping the {} queue partitions in order to change node identifiers from {} to {}", this, queuePartitions.length, this.nodeIdentifiers, updatedNodeIdentifiers); for (final QueuePartition queuePartition : queuePartitions) { queuePartition.stop(); @@ -968,6 +988,20 @@ public void onNodeAdded(final NodeIdentifier nodeId) { } } + @Override + public void onNodeOffloaded(final NodeIdentifier nodeId) { + partitionWriteLock.lock(); + try { + final Set updatedNodeIds = new HashSet<>(nodeIdentifiers); + updatedNodeIds.remove(nodeId); + + logger.debug("Node Identifier {} offloaded. Node ID's changing from {} to {}", nodeId, nodeIdentifiers, updatedNodeIds); + setNodeIdentifiers(updatedNodeIds, false); + } finally { + partitionWriteLock.unlock(); + } + } + @Override public void onNodeRemoved(final NodeIdentifier nodeId) { partitionWriteLock.lock(); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/partition/NonLocalPartitionPartitioner.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/partition/NonLocalPartitionPartitioner.java new file mode 100644 index 000000000000..cffaefd5cbba --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/partition/NonLocalPartitionPartitioner.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.controller.queue.clustered.partition; + +import org.apache.nifi.controller.repository.FlowFileRecord; + +import java.util.concurrent.atomic.AtomicLong; + +/** + * Returns remote partitions when queried for a partition; never returns the {@link LocalQueuePartition}. + */ +public class NonLocalPartitionPartitioner implements FlowFilePartitioner { + private final AtomicLong counter = new AtomicLong(0L); + + @Override + public QueuePartition getPartition(final FlowFileRecord flowFile, final QueuePartition[] partitions, final QueuePartition localPartition) { + QueuePartition remotePartition = null; + final long startIndex = counter.getAndIncrement(); + for (int i = 0, numPartitions = partitions.length; i < numPartitions && remotePartition == null; ++i) { + int index = (int) ((startIndex + i) % numPartitions); + QueuePartition partition = partitions[index]; + if (!partition.equals(localPartition)) { + remotePartition = partition; + } + } + + if (remotePartition == null) { + throw new IllegalStateException("Could not determine a remote partition"); + } + + return remotePartition; + } + + @Override + public boolean isRebalanceOnClusterResize() { + return false; + } + + + @Override + public boolean isRebalanceOnFailure() { + return true; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java index 402ce06ad16c..878ad130d42c 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java @@ -109,6 +109,10 @@ public void startLoadBalancing() { public void stopLoadBalancing() { } + @Override + public void offloadQueue() { + } + @Override public boolean isActivelyLoadBalancing() { return false; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java index b8ac88a58bfe..4ef241e8f59c 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java @@ -47,6 +47,7 @@ import org.apache.nifi.cluster.coordination.heartbeat.HeartbeatMonitor; import org.apache.nifi.cluster.coordination.heartbeat.NodeHeartbeat; import org.apache.nifi.cluster.coordination.node.ClusterRoles; +import org.apache.nifi.cluster.coordination.node.OffloadCode; import org.apache.nifi.cluster.coordination.node.DisconnectionCode; import org.apache.nifi.cluster.coordination.node.NodeConnectionState; import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; @@ -1124,6 +1125,9 @@ public NodeDTO updateNode(final NodeDTO nodeDTO) { if (NodeConnectionState.CONNECTING.name().equalsIgnoreCase(nodeDTO.getStatus())) { clusterCoordinator.requestNodeConnect(nodeId, userDn); + } else if (NodeConnectionState.OFFLOADING.name().equalsIgnoreCase(nodeDTO.getStatus())) { + clusterCoordinator.requestNodeOffload(nodeId, OffloadCode.OFFLOADED, + "User " + userDn + " requested that node be offloaded"); } else if (NodeConnectionState.DISCONNECTING.name().equalsIgnoreCase(nodeDTO.getStatus())) { clusterCoordinator.requestNodeDisconnect(nodeId, DisconnectionCode.USER_DISCONNECTED, "User " + userDn + " requested that node be disconnected from cluster"); @@ -4702,7 +4706,7 @@ public void deleteNode(final String nodeId) { } final NodeConnectionStatus nodeConnectionStatus = clusterCoordinator.getConnectionStatus(nodeIdentifier); - if (!nodeConnectionStatus.getState().equals(NodeConnectionState.DISCONNECTED)) { + if (!nodeConnectionStatus.getState().equals(NodeConnectionState.OFFLOADED) && !nodeConnectionStatus.getState().equals(NodeConnectionState.DISCONNECTED)) { throw new IllegalNodeDeletionException("Cannot remove Node with ID " + nodeId + " because it is not disconnected, current state = " + nodeConnectionStatus.getState()); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/config/IllegalNodeOffloadExceptionMapper.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/config/IllegalNodeOffloadExceptionMapper.java new file mode 100644 index 000000000000..890a8a6726af --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/config/IllegalNodeOffloadExceptionMapper.java @@ -0,0 +1,46 @@ +/* + * 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.web.api.config; + +import org.apache.nifi.cluster.manager.exception.IllegalNodeOffloadException; +import org.apache.nifi.util.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.ws.rs.core.Response; +import javax.ws.rs.ext.ExceptionMapper; +import javax.ws.rs.ext.Provider; + +/** + * Maps illegal node offload exceptions into client responses. + */ +@Provider +public class IllegalNodeOffloadExceptionMapper implements ExceptionMapper { + + private static final Logger logger = LoggerFactory.getLogger(IllegalNodeOffloadExceptionMapper.class); + + @Override + public Response toResponse(IllegalNodeOffloadException exception) { + logger.info(String.format("%s. Returning %s response.", exception, Response.Status.CONFLICT)); + + if (logger.isDebugEnabled()) { + logger.debug(StringUtils.EMPTY, exception); + } + return Response.status(Response.Status.CONFLICT).entity(exception.getMessage()).type("text/plain").build(); + } + +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/frontend/package.json b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/frontend/package.json index eff9e33ce8d1..0bbd0ed00c6c 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/frontend/package.json +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/frontend/package.json @@ -30,7 +30,7 @@ "d3-selection-multi": "1.0.1", "jquery-minicolors": "2.1.10", "jquery-ui-dist": "1.12.1", - "font-awesome": "4.6.1", + "font-awesome": "4.7.0", "jquery": "3.1.1", "reset.css": "2.0.2", "jquery-form": "3.50.0", diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/cluster/nf-cluster-table.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/cluster/nf-cluster-table.js index 095f7149f4ed..0dc74d7cad5a 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/cluster/nf-cluster-table.js +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/cluster/nf-cluster-table.js @@ -528,6 +528,8 @@ promptForConnect(item); } else if (target.hasClass('prompt-for-removal')) { promptForRemoval(item); + } else if (target.hasClass('prompt-for-offload')) { + promptForOffload(item); } else if (target.hasClass('prompt-for-disconnect')) { promptForDisconnect(item); } @@ -630,19 +632,29 @@ var actionFormatter = function (row, cell, value, columnDef, dataContext) { var canDisconnect = false; var canConnect = false; + var isOffloaded = false; // determine the current status if (dataContext.status === 'CONNECTED' || dataContext.status === 'CONNECTING') { canDisconnect = true; - } else if (dataContext.status === 'DISCONNECTED') { + } + if (dataContext.status === 'DISCONNECTED') { canConnect = true; } + if (dataContext.status === 'OFFLOADED') { + isOffloaded = true; + } // return the appropriate markup if (canConnect) { - return '
'; + return '
' + + '
' + + '
'; } else if (canDisconnect) { return '
'; + } else if (isOffloaded) { + return '
' + + '
'; } else { return '
 
'; } @@ -946,6 +958,50 @@ }).fail(nfErrorHandler.handleAjaxError); }; + /** + * Prompts to verify node offload. + * + * @argument {object} node The node + */ + var promptForOffload = function (node) { + nfDialog.showYesNoDialog({ + headerText: 'Offload Node', + dialogContent: 'Offload \'' + formatNodeAddress(node) + '\'?', + yesHandler: function () { + offload(node.nodeId); + } + }); + }; + + /** + * Offloads the node in the specified row. + * + * @argument {string} nodeId The node id + */ + var offload = function (nodeId) { + var entity = { + 'node': { + 'nodeId': nodeId, + 'status': 'OFFLOADING' + } + }; + + $.ajax({ + type: 'PUT', + url: config.urls.nodes + '/' + encodeURIComponent(nodeId), + data: JSON.stringify(entity), + dataType: 'json', + contentType: 'application/json' + }).done(function (response) { + var node = response.node; + + // update the node in the table + var clusterGrid = $('#cluster-nodes-table').data('gridInstance'); + var clusterData = clusterGrid.getData(); + clusterData.updateItem(node.nodeId, node); + }).fail(nfErrorHandler.handleAjaxError); + }; + /** * Prompts to verify node disconnection. * diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerClient.java index 22821ee3845e..6cb32264957b 100644 --- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerClient.java +++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerClient.java @@ -16,6 +16,8 @@ */ package org.apache.nifi.toolkit.cli.impl.client.nifi; +import org.apache.nifi.web.api.entity.ClusterEntity; +import org.apache.nifi.web.api.entity.NodeEntity; import org.apache.nifi.web.api.entity.RegistryClientEntity; import org.apache.nifi.web.api.entity.RegistryClientsEntity; @@ -34,4 +36,16 @@ public interface ControllerClient { RegistryClientEntity updateRegistryClient(RegistryClientEntity registryClientEntity) throws NiFiClientException, IOException; + NodeEntity connectNode(String nodeId, NodeEntity nodeEntity) throws NiFiClientException, IOException; + + NodeEntity deleteNode(String nodeId) throws NiFiClientException, IOException; + + NodeEntity disconnectNode(String nodeId, NodeEntity nodeEntity) throws NiFiClientException, IOException; + + NodeEntity getNode(String nodeId) throws NiFiClientException, IOException; + + ClusterEntity getNodes() throws NiFiClientException, IOException; + + NodeEntity offloadNode(String nodeId, NodeEntity nodeEntity) throws NiFiClientException, IOException; + } diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyControllerClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyControllerClient.java index 9c9ffc49d339..a16279046891 100644 --- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyControllerClient.java +++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyControllerClient.java @@ -19,6 +19,8 @@ import org.apache.commons.lang3.StringUtils; import org.apache.nifi.toolkit.cli.impl.client.nifi.ControllerClient; import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException; +import org.apache.nifi.web.api.entity.ClusterEntity; +import org.apache.nifi.web.api.entity.NodeEntity; import org.apache.nifi.web.api.entity.RegistryClientEntity; import org.apache.nifi.web.api.entity.RegistryClientsEntity; @@ -104,4 +106,89 @@ public RegistryClientEntity updateRegistryClient(final RegistryClientEntity regi }); } + @Override + public NodeEntity deleteNode(final String nodeId) throws NiFiClientException, IOException { + if (StringUtils.isBlank(nodeId)) { + throw new IllegalArgumentException("Node ID cannot be null or empty"); + } + + return executeAction("Error deleting node", () -> { + final WebTarget target = controllerTarget.path("cluster/nodes/" + nodeId); + + return getRequestBuilder(target).delete(NodeEntity.class); + }); + } + + @Override + public NodeEntity connectNode(final String nodeId, final NodeEntity nodeEntity) throws NiFiClientException, IOException { + if (StringUtils.isBlank(nodeId)) { + throw new IllegalArgumentException("Node ID cannot be null or empty"); + } + + if (nodeEntity == null) { + throw new IllegalArgumentException("Node entity cannot be null"); + } + + return executeAction("Error connecting node", () -> { + final WebTarget target = controllerTarget.path("cluster/nodes/" + nodeId); + + return getRequestBuilder(target).put(Entity.entity(nodeEntity, MediaType.APPLICATION_JSON), NodeEntity.class); + }); + } + + @Override + public NodeEntity offloadNode(final String nodeId, final NodeEntity nodeEntity) throws NiFiClientException, IOException { + if (StringUtils.isBlank(nodeId)) { + throw new IllegalArgumentException("Node ID cannot be null or empty"); + } + + if (nodeEntity == null) { + throw new IllegalArgumentException("Node entity cannot be null"); + } + + return executeAction("Error offloading node", () -> { + final WebTarget target = controllerTarget.path("cluster/nodes/" + nodeId); + + return getRequestBuilder(target).put(Entity.entity(nodeEntity, MediaType.APPLICATION_JSON), NodeEntity.class); + }); + } + + @Override + public NodeEntity disconnectNode(final String nodeId, final NodeEntity nodeEntity) throws NiFiClientException, IOException { + if (StringUtils.isBlank(nodeId)) { + throw new IllegalArgumentException("Node ID cannot be null or empty"); + } + + if (nodeEntity == null) { + throw new IllegalArgumentException("Node entity cannot be null"); + } + + return executeAction("Error disconnecting node", () -> { + final WebTarget target = controllerTarget.path("cluster/nodes/" + nodeId); + + return getRequestBuilder(target).put(Entity.entity(nodeEntity, MediaType.APPLICATION_JSON), NodeEntity.class); + }); + } + + @Override + public NodeEntity getNode(String nodeId) throws NiFiClientException, IOException { + if (StringUtils.isBlank(nodeId)) { + throw new IllegalArgumentException("Node ID cannot be null or empty"); + } + + return executeAction("Error retrieving node status", () -> { + final WebTarget target = controllerTarget.path("cluster/nodes/" + nodeId); + + return getRequestBuilder(target).get(NodeEntity.class); + }); + } + + @Override + public ClusterEntity getNodes() throws NiFiClientException, IOException { + return executeAction("Error retrieving node status", () -> { + final WebTarget target = controllerTarget.path("cluster"); + + return getRequestBuilder(target).get(ClusterEntity.class); + }); + } } diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/CommandOption.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/CommandOption.java index ad15036f3b9f..171e6cf85eb8 100644 --- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/CommandOption.java +++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/CommandOption.java @@ -49,6 +49,9 @@ public enum CommandOption { SRC_FLOW_ID("sf", "sourceFlowIdentifier", "A flow identifier from the source registry", true), SRC_FLOW_VERSION("sfv", "sourceFlowVersion", "A version of a flow from the source registry", true), + // NiFi - Nodes + NIFI_NODE_ID("nnid", "nifiNodeId", "The ID of a node in the NiFi cluster", true), + // NiFi - Registries REGISTRY_CLIENT_ID("rcid", "registryClientId", "The id of a registry client", true), REGISTRY_CLIENT_NAME("rcn", "registryClientName", "The name of the registry client", true), diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/NiFiCommandGroup.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/NiFiCommandGroup.java index 00a38a222c2f..298b709fee1b 100644 --- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/NiFiCommandGroup.java +++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/NiFiCommandGroup.java @@ -21,6 +21,12 @@ import org.apache.nifi.toolkit.cli.impl.command.nifi.flow.ClusterSummary; import org.apache.nifi.toolkit.cli.impl.command.nifi.flow.CurrentUser; import org.apache.nifi.toolkit.cli.impl.command.nifi.flow.GetRootId; +import org.apache.nifi.toolkit.cli.impl.command.nifi.nodes.ConnectNode; +import org.apache.nifi.toolkit.cli.impl.command.nifi.nodes.OffloadNode; +import org.apache.nifi.toolkit.cli.impl.command.nifi.nodes.DeleteNode; +import org.apache.nifi.toolkit.cli.impl.command.nifi.nodes.DisconnectNode; +import org.apache.nifi.toolkit.cli.impl.command.nifi.nodes.GetNode; +import org.apache.nifi.toolkit.cli.impl.command.nifi.nodes.GetNodes; import org.apache.nifi.toolkit.cli.impl.command.nifi.pg.PGChangeVersion; import org.apache.nifi.toolkit.cli.impl.command.nifi.pg.PGDisableControllerServices; import org.apache.nifi.toolkit.cli.impl.command.nifi.pg.PGEnableControllerServices; @@ -58,7 +64,13 @@ protected List createCommands() { final List commands = new ArrayList<>(); commands.add(new CurrentUser()); commands.add(new ClusterSummary()); + commands.add(new ConnectNode()); + commands.add(new DeleteNode()); + commands.add(new DisconnectNode()); commands.add(new GetRootId()); + commands.add(new GetNode()); + commands.add(new GetNodes()); + commands.add(new OffloadNode()); commands.add(new ListRegistryClients()); commands.add(new CreateRegistryClient()); commands.add(new UpdateRegistryClient()); diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/nodes/ConnectNode.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/nodes/ConnectNode.java new file mode 100644 index 000000000000..8ec006692bae --- /dev/null +++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/nodes/ConnectNode.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.toolkit.cli.impl.command.nifi.nodes; + +import org.apache.commons.cli.MissingOptionException; +import org.apache.nifi.toolkit.cli.api.CommandException; +import org.apache.nifi.toolkit.cli.api.Context; +import org.apache.nifi.toolkit.cli.impl.client.nifi.ControllerClient; +import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient; +import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException; +import org.apache.nifi.toolkit.cli.impl.command.CommandOption; +import org.apache.nifi.toolkit.cli.impl.command.nifi.AbstractNiFiCommand; +import org.apache.nifi.toolkit.cli.impl.result.NodeResult; +import org.apache.nifi.web.api.dto.NodeDTO; +import org.apache.nifi.web.api.entity.NodeEntity; + +import java.io.IOException; +import java.util.Properties; + +/** + * Command for offloading a node of the NiFi cluster. + */ +public class ConnectNode extends AbstractNiFiCommand { + + public ConnectNode() { + super("connect-node", NodeResult.class); + } + + @Override + public String getDescription() { + return "Connects a node to the NiFi cluster."; + } + + @Override + protected void doInitialize(Context context) { + addOption(CommandOption.NIFI_NODE_ID.createOption()); + } + + @Override + public NodeResult doExecute(NiFiClient client, Properties properties) throws NiFiClientException, IOException, MissingOptionException, CommandException { + final String nodeId = getRequiredArg(properties, CommandOption.NIFI_NODE_ID); + final ControllerClient controllerClient = client.getControllerClient(); + + NodeDTO nodeDto = new NodeDTO(); + nodeDto.setNodeId(nodeId); + // TODO There are no constants for the CONNECT node statuses + nodeDto.setStatus("CONNECTING"); + NodeEntity nodeEntity = new NodeEntity(); + nodeEntity.setNode(nodeDto); + NodeEntity nodeEntityResult = controllerClient.connectNode(nodeId, nodeEntity); + return new NodeResult(getResultType(properties), nodeEntityResult); + } +} diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/nodes/DeleteNode.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/nodes/DeleteNode.java new file mode 100644 index 000000000000..280e625fd94c --- /dev/null +++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/nodes/DeleteNode.java @@ -0,0 +1,58 @@ +/* + * 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.toolkit.cli.impl.command.nifi.nodes; + +import org.apache.commons.cli.MissingOptionException; +import org.apache.nifi.toolkit.cli.api.Context; +import org.apache.nifi.toolkit.cli.impl.client.nifi.ControllerClient; +import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient; +import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException; +import org.apache.nifi.toolkit.cli.impl.command.CommandOption; +import org.apache.nifi.toolkit.cli.impl.command.nifi.AbstractNiFiCommand; +import org.apache.nifi.toolkit.cli.impl.result.OkResult; + +import java.io.IOException; +import java.util.Properties; + +/** + * Command for deleting a node from the NiFi cluster. + */ +public class DeleteNode extends AbstractNiFiCommand { + + public DeleteNode() { + super("delete-node", OkResult.class); + } + + @Override + public String getDescription() { + return "Deletes a node from the NiFi cluster."; + } + + @Override + protected void doInitialize(Context context) { + addOption(CommandOption.NIFI_NODE_ID.createOption()); + } + + @Override + public OkResult doExecute(NiFiClient client, Properties properties) throws NiFiClientException, IOException, MissingOptionException { + final String nodeId = getRequiredArg(properties, CommandOption.NIFI_NODE_ID); + final ControllerClient controllerClient = client.getControllerClient(); + + controllerClient.deleteNode(nodeId); + return new OkResult(getContext().isInteractive()); + } +} diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/nodes/DisconnectNode.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/nodes/DisconnectNode.java new file mode 100644 index 000000000000..98fa03a11541 --- /dev/null +++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/nodes/DisconnectNode.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.toolkit.cli.impl.command.nifi.nodes; + +import org.apache.commons.cli.MissingOptionException; +import org.apache.nifi.toolkit.cli.api.CommandException; +import org.apache.nifi.toolkit.cli.api.Context; +import org.apache.nifi.toolkit.cli.impl.client.nifi.ControllerClient; +import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient; +import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException; +import org.apache.nifi.toolkit.cli.impl.command.CommandOption; +import org.apache.nifi.toolkit.cli.impl.command.nifi.AbstractNiFiCommand; +import org.apache.nifi.toolkit.cli.impl.result.NodeResult; +import org.apache.nifi.web.api.dto.NodeDTO; +import org.apache.nifi.web.api.entity.NodeEntity; + +import java.io.IOException; +import java.util.Properties; + +/** + * Command for disconnecting a node from the NiFi cluster. + */ +public class DisconnectNode extends AbstractNiFiCommand { + + public DisconnectNode() { + super("disconnect-node", NodeResult.class); + } + + @Override + public String getDescription() { + return "Disconnects a node from the NiFi cluster."; + } + + @Override + protected void doInitialize(Context context) { + addOption(CommandOption.NIFI_NODE_ID.createOption()); + } + + @Override + public NodeResult doExecute(NiFiClient client, Properties properties) throws NiFiClientException, IOException, MissingOptionException, CommandException { + final String nodeId = getRequiredArg(properties, CommandOption.NIFI_NODE_ID); + final ControllerClient controllerClient = client.getControllerClient(); + + NodeDTO nodeDto = new NodeDTO(); + nodeDto.setNodeId(nodeId); + // TODO There's no constant for node status in + nodeDto.setStatus("DISCONNECTING"); + NodeEntity nodeEntity = new NodeEntity(); + nodeEntity.setNode(nodeDto); + NodeEntity nodeEntityResult = controllerClient.disconnectNode(nodeId, nodeEntity); + return new NodeResult(getResultType(properties), nodeEntityResult); + } +} diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/nodes/GetNode.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/nodes/GetNode.java new file mode 100644 index 000000000000..54687bdb124c --- /dev/null +++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/nodes/GetNode.java @@ -0,0 +1,59 @@ +/* + * 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.toolkit.cli.impl.command.nifi.nodes; + +import org.apache.commons.cli.MissingOptionException; +import org.apache.nifi.toolkit.cli.api.Context; +import org.apache.nifi.toolkit.cli.impl.client.nifi.ControllerClient; +import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient; +import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException; +import org.apache.nifi.toolkit.cli.impl.command.CommandOption; +import org.apache.nifi.toolkit.cli.impl.command.nifi.AbstractNiFiCommand; +import org.apache.nifi.toolkit.cli.impl.result.NodeResult; +import org.apache.nifi.web.api.entity.NodeEntity; + +import java.io.IOException; +import java.util.Properties; + +/** + * Command for retrieving the status of the nodes from the NiFi cluster. + */ +public class GetNode extends AbstractNiFiCommand { + + public GetNode() { + super("get-node", NodeResult.class); + } + + @Override + public String getDescription() { + return "Retrieves the status for a node in the NiFi cluster."; + } + + @Override + protected void doInitialize(Context context) { + addOption(CommandOption.NIFI_NODE_ID.createOption()); + } + + @Override + public NodeResult doExecute(NiFiClient client, Properties properties) throws NiFiClientException, IOException, MissingOptionException { + final String nodeId = getRequiredArg(properties, CommandOption.NIFI_NODE_ID); + final ControllerClient controllerClient = client.getControllerClient(); + + NodeEntity nodeEntityResult = controllerClient.getNode(nodeId); + return new NodeResult(getResultType(properties), nodeEntityResult); + } +} diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/nodes/GetNodes.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/nodes/GetNodes.java new file mode 100644 index 000000000000..368fb4ddab4c --- /dev/null +++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/nodes/GetNodes.java @@ -0,0 +1,52 @@ +/* + * 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.toolkit.cli.impl.command.nifi.nodes; + +import org.apache.commons.cli.MissingOptionException; +import org.apache.nifi.toolkit.cli.api.CommandException; +import org.apache.nifi.toolkit.cli.impl.client.nifi.ControllerClient; +import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient; +import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException; +import org.apache.nifi.toolkit.cli.impl.command.nifi.AbstractNiFiCommand; +import org.apache.nifi.toolkit.cli.impl.result.NodesResult; +import org.apache.nifi.web.api.entity.ClusterEntity; + +import java.io.IOException; +import java.util.Properties; + +/** + * Command for retrieving the status of the nodes from the NiFi cluster. + */ +public class GetNodes extends AbstractNiFiCommand { + + public GetNodes() { + super("get-nodes", NodesResult.class); + } + + @Override + public String getDescription() { + return "Retrieves statuses for the nodes of the NiFi cluster."; + } + + @Override + public NodesResult doExecute(NiFiClient client, Properties properties) throws NiFiClientException, IOException, MissingOptionException, CommandException { + final ControllerClient controllerClient = client.getControllerClient(); + + ClusterEntity clusterEntityResult = controllerClient.getNodes(); + return new NodesResult(getResultType(properties), clusterEntityResult); + } +} diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/nodes/OffloadNode.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/nodes/OffloadNode.java new file mode 100644 index 000000000000..aa759b1c7dcf --- /dev/null +++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/nodes/OffloadNode.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.toolkit.cli.impl.command.nifi.nodes; + +import org.apache.commons.cli.MissingOptionException; +import org.apache.nifi.toolkit.cli.api.CommandException; +import org.apache.nifi.toolkit.cli.api.Context; +import org.apache.nifi.toolkit.cli.impl.client.nifi.ControllerClient; +import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient; +import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException; +import org.apache.nifi.toolkit.cli.impl.command.CommandOption; +import org.apache.nifi.toolkit.cli.impl.command.nifi.AbstractNiFiCommand; +import org.apache.nifi.toolkit.cli.impl.result.NodeResult; +import org.apache.nifi.web.api.dto.NodeDTO; +import org.apache.nifi.web.api.entity.NodeEntity; + +import java.io.IOException; +import java.util.Properties; + +/** + * Command for offloading a node of the NiFi cluster. + */ +public class OffloadNode extends AbstractNiFiCommand { + + public OffloadNode() { + super("offload-node", NodeResult.class); + } + + @Override + public String getDescription() { + return "Offloads a node of the NiFi cluster."; + } + + @Override + protected void doInitialize(Context context) { + addOption(CommandOption.NIFI_NODE_ID.createOption()); + } + + @Override + public NodeResult doExecute(NiFiClient client, Properties properties) throws NiFiClientException, IOException, MissingOptionException, CommandException { + final String nodeId = getRequiredArg(properties, CommandOption.NIFI_NODE_ID); + final ControllerClient controllerClient = client.getControllerClient(); + + NodeDTO nodeDto = new NodeDTO(); + nodeDto.setNodeId(nodeId); + // TODO There are no constants for the OFFLOAD node statuses + nodeDto.setStatus("OFFLOADING"); + NodeEntity nodeEntity = new NodeEntity(); + nodeEntity.setNode(nodeDto); + NodeEntity nodeEntityResult = controllerClient.offloadNode(nodeId, nodeEntity); + return new NodeResult(getResultType(properties), nodeEntityResult); + } +} diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/result/NodeResult.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/result/NodeResult.java new file mode 100644 index 000000000000..3e1efdf46eea --- /dev/null +++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/result/NodeResult.java @@ -0,0 +1,48 @@ +/* + * 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.toolkit.cli.impl.result; + +import org.apache.commons.lang3.Validate; +import org.apache.nifi.toolkit.cli.api.ResultType; +import org.apache.nifi.web.api.dto.NodeDTO; +import org.apache.nifi.web.api.entity.NodeEntity; + +import java.io.IOException; +import java.io.PrintStream; + +public class NodeResult extends AbstractWritableResult { + + private final NodeEntity nodeEntity; + + public NodeResult(ResultType resultType, NodeEntity nodeEntity) { + super(resultType); + this.nodeEntity = nodeEntity; + Validate.notNull(nodeEntity); + } + + @Override + public NodeEntity getResult() { + return nodeEntity; + } + + @Override + protected void writeSimpleResult(PrintStream output) throws IOException { + NodeDTO nodeDTO = nodeEntity.getNode(); + output.printf("Node ID: %s\nNode Address: %s\nAPI Port: %s\nNode Status:%s", + nodeDTO.getNodeId(), nodeDTO.getAddress(), nodeDTO.getApiPort(), nodeDTO.getStatus()); + } +} diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/result/NodesResult.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/result/NodesResult.java new file mode 100644 index 000000000000..daab27fd82e6 --- /dev/null +++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/result/NodesResult.java @@ -0,0 +1,66 @@ +/* + * 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.toolkit.cli.impl.result; + +import org.apache.commons.lang3.Validate; +import org.apache.nifi.toolkit.cli.api.ResultType; +import org.apache.nifi.toolkit.cli.impl.result.writer.DynamicTableWriter; +import org.apache.nifi.toolkit.cli.impl.result.writer.Table; +import org.apache.nifi.toolkit.cli.impl.result.writer.TableWriter; +import org.apache.nifi.web.api.dto.NodeDTO; +import org.apache.nifi.web.api.entity.ClusterEntity; +import org.glassfish.jersey.internal.guava.Lists; + +import java.io.IOException; +import java.io.PrintStream; +import java.util.List; + +public class NodesResult extends AbstractWritableResult { + + private final ClusterEntity clusterEntity; + + public NodesResult(ResultType resultType, ClusterEntity clusterEntity) { + super(resultType); + this.clusterEntity = clusterEntity; + Validate.notNull(clusterEntity); + } + + @Override + public ClusterEntity getResult() { + return clusterEntity; + } + + @Override + protected void writeSimpleResult(PrintStream output) throws IOException { + final Table table = new Table.Builder() + .column("#", 3, 3, false) + .column("Node ID", 36, 36, false) + .column("Node Address", 36, 36, true) + .column("API Port", 8, 8, false) + .column("Node Status", 13, 13, false) + .build(); + + List nodes = Lists.newArrayList(clusterEntity.getCluster().getNodes()); + for (int i = 0; i < nodes.size(); ++i) { + NodeDTO nodeDTO = nodes.get(i); + table.addRow(String.valueOf(i), nodeDTO.getNodeId(), nodeDTO.getAddress(), String.valueOf(nodeDTO.getApiPort()), nodeDTO.getStatus()); + } + + final TableWriter tableWriter = new DynamicTableWriter(); + tableWriter.write(table, output); + } +} From e3856df3f8884730d94c1c100bf114eefba2e90e Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Mon, 24 Sep 2018 09:17:22 -0400 Subject: [PATCH 2/5] NIFI-5585: Fixed bug that arised when multiple nodes were decommissioning at same time; could get into state where the nodes queued up data for one another so the data just stayed put --- .../ClusterTopologyEventListener.java | 5 +- .../node/NodeClusterCoordinator.java | 15 +- .../SocketLoadBalancedFlowFileQueue.java | 130 ++++++++++++------ .../NonLocalPartitionPartitioner.java | 2 +- 4 files changed, 101 insertions(+), 51 deletions(-) diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/ClusterTopologyEventListener.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/ClusterTopologyEventListener.java index ad9be3d738e5..d31339b91a32 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/ClusterTopologyEventListener.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/ClusterTopologyEventListener.java @@ -17,15 +17,16 @@ package org.apache.nifi.cluster.coordination; +import org.apache.nifi.cluster.coordination.node.NodeConnectionState; import org.apache.nifi.cluster.protocol.NodeIdentifier; public interface ClusterTopologyEventListener { void onNodeAdded(NodeIdentifier nodeId); - void onNodeOffloaded(NodeIdentifier nodeId); - void onNodeRemoved(NodeIdentifier nodeId); void onLocalNodeIdentifierSet(NodeIdentifier localNodeId); + + void onNodeStateChange(NodeIdentifier nodeId, NodeConnectionState newState); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java index 65a0a390d843..d697df122396 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java @@ -341,6 +341,8 @@ private NodeConnectionStatus updateNodeStatus(final NodeIdentifier nodeId, final final NodeConnectionStatus evictedStatus = nodeStatuses.put(nodeId, updatedStatus); if (evictedStatus == null) { onNodeAdded(nodeId, storeState); + } else { + onNodeStateChange(nodeId, updatedStatus.getState()); } return evictedStatus; @@ -359,6 +361,10 @@ private boolean updateNodeStatusConditionally(final NodeIdentifier nodeId, final updated = nodeStatuses.replace(nodeId, expectedStatus, updatedStatus); } + if (updated) { + onNodeStateChange(nodeId, updatedStatus.getState()); + } + return updated; } @@ -511,7 +517,6 @@ public void requestNodeOffload(final NodeIdentifier nodeId, final OffloadCode of request.setExplanation(explanation); addNodeEvent(nodeId, "Offload requested due to " + explanation); - onNodeOffloaded(nodeId); offloadAsynchronously(request, 10, 5); } @@ -572,10 +577,6 @@ public void removeNode(final NodeIdentifier nodeId, final String userDn) { storeState(); } - private void onNodeOffloaded(final NodeIdentifier nodeId) { - eventListeners.forEach(listener -> listener.onNodeOffloaded(nodeId)); - } - private void onNodeRemoved(final NodeIdentifier nodeId) { eventListeners.forEach(listener -> listener.onNodeRemoved(nodeId)); } @@ -587,6 +588,10 @@ private void onNodeAdded(final NodeIdentifier nodeId, final boolean storeState) eventListeners.forEach(listener -> listener.onNodeAdded(nodeId)); } + private void onNodeStateChange(final NodeIdentifier nodeId, final NodeConnectionState nodeConnectionState) { + eventListeners.forEach(listener -> listener.onNodeStateChange(nodeId, nodeConnectionState)); + } + @Override public NodeConnectionStatus getConnectionStatus(final NodeIdentifier nodeId) { return nodeStatuses.get(nodeId); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/SocketLoadBalancedFlowFileQueue.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/SocketLoadBalancedFlowFileQueue.java index 3143332893e9..8ee072060bc7 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/SocketLoadBalancedFlowFileQueue.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/SocketLoadBalancedFlowFileQueue.java @@ -19,6 +19,8 @@ import org.apache.nifi.cluster.coordination.ClusterCoordinator; import org.apache.nifi.cluster.coordination.ClusterTopologyEventListener; +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.controller.ProcessScheduler; import org.apache.nifi.controller.queue.AbstractFlowFileQueue; @@ -114,7 +116,7 @@ public class SocketLoadBalancedFlowFileQueue extends AbstractFlowFileQueue imple private QueuePartition[] queuePartitions; private FlowFilePartitioner partitioner; private boolean stopped = true; - private boolean offloaded = false; + private volatile boolean offloaded = false; public SocketLoadBalancedFlowFileQueue(final String identifier, final ConnectionEventListener eventListener, final ProcessScheduler scheduler, final FlowFileRepository flowFileRepo, @@ -184,26 +186,28 @@ public synchronized void setLoadBalanceStrategy(final LoadBalanceStrategy strate return; } - // We are already load balancing but are changing how we are load balancing. - final FlowFilePartitioner partitioner; - switch (strategy) { - case DO_NOT_LOAD_BALANCE: - partitioner = new LocalPartitionPartitioner(); - break; - case PARTITION_BY_ATTRIBUTE: - partitioner = new CorrelationAttributePartitioner(partitioningAttribute); - break; - case ROUND_ROBIN: - partitioner = new RoundRobinPartitioner(); - break; - case SINGLE_NODE: - partitioner = new FirstNodePartitioner(); - break; - default: - throw new IllegalArgumentException(); - } + if (!offloaded) { + // We are already load balancing but are changing how we are load balancing. + final FlowFilePartitioner partitioner; + switch (strategy) { + case DO_NOT_LOAD_BALANCE: + partitioner = new LocalPartitionPartitioner(); + break; + case PARTITION_BY_ATTRIBUTE: + partitioner = new CorrelationAttributePartitioner(partitioningAttribute); + break; + case ROUND_ROBIN: + partitioner = new RoundRobinPartitioner(); + break; + case SINGLE_NODE: + partitioner = new FirstNodePartitioner(); + break; + default: + throw new IllegalArgumentException(); + } - setFlowFilePartitioner(partitioner); + setFlowFilePartitioner(partitioner); + } } @Override @@ -215,8 +219,33 @@ public void offloadQueue() { offloaded = true; - // TODO need to be able to reset the partitioner to the previous partitioner if this node is reconnected to the cluster - setFlowFilePartitioner(new NonLocalPartitionPartitioner()); + partitionWriteLock.lock(); + try { + final Set nodesToKeep = new HashSet<>(); + + // If we have any nodes that are connected, we only want to send data to the connected nodes. + for (final QueuePartition partition : queuePartitions) { + final Optional nodeIdOption = partition.getNodeIdentifier(); + if (!nodeIdOption.isPresent()) { + continue; + } + + final NodeIdentifier nodeId = nodeIdOption.get(); + final NodeConnectionStatus status = clusterCoordinator.getConnectionStatus(nodeId); + if (status != null && status.getState() == NodeConnectionState.CONNECTED) { + nodesToKeep.add(nodeId); + } + } + + if (!nodesToKeep.isEmpty()) { + setNodeIdentifiers(nodesToKeep, false); + } + + // Update our partitioner so that we don't keep any data on the local partition + setFlowFilePartitioner(new NonLocalPartitionPartitioner()); + } finally { + partitionWriteLock.unlock(); + } } public synchronized void startLoadBalancing() { @@ -566,11 +595,6 @@ public void setNodeIdentifiers(final Set updatedNodeIdentifiers, return; } - if (offloaded) { - logger.debug("{} Not going to rebalance Queue even though setNodeIdentifiers was called, because the queue has been offloaded", this); - return; - } - logger.debug("{} Stopping the {} queue partitions in order to change node identifiers from {} to {}", this, queuePartitions.length, this.nodeIdentifiers, updatedNodeIdentifiers); for (final QueuePartition queuePartition : queuePartitions) { queuePartition.stop(); @@ -593,7 +617,7 @@ public void setNodeIdentifiers(final Set updatedNodeIdentifiers, final List sortedNodeIdentifiers = new ArrayList<>(updatedNodeIdentifiers); sortedNodeIdentifiers.sort(Comparator.comparing(NodeIdentifier::getApiAddress)); - final QueuePartition[] updatedQueuePartitions; + QueuePartition[] updatedQueuePartitions; if (sortedNodeIdentifiers.isEmpty()) { updatedQueuePartitions = new QueuePartition[] { localPartition }; } else { @@ -601,10 +625,12 @@ public void setNodeIdentifiers(final Set updatedNodeIdentifiers, } // Populate the new QueuePartitions. + boolean localPartitionIncluded = false; for (int i = 0; i < sortedNodeIdentifiers.size(); i++) { final NodeIdentifier nodeId = sortedNodeIdentifiers.get(i); if (nodeId.equals(clusterCoordinator.getLocalNodeIdentifier())) { updatedQueuePartitions[i] = localPartition; + localPartitionIncluded = true; // If we have RemoteQueuePartition with this Node ID with data, that data must be migrated to the local partition. // This can happen if we didn't previously know our Node UUID. @@ -622,6 +648,13 @@ public void setNodeIdentifiers(final Set updatedNodeIdentifiers, updatedQueuePartitions[i] = existingPartition == null ? createRemotePartition(nodeId) : existingPartition; } + if (!localPartitionIncluded) { + final QueuePartition[] withLocal = new QueuePartition[updatedQueuePartitions.length + 1]; + System.arraycopy(updatedQueuePartitions, 0, withLocal, 0, updatedQueuePartitions.length); + withLocal[withLocal.length - 1] = localPartition; + updatedQueuePartitions = withLocal; + } + // If the partition requires that all partitions be re-balanced when the number of partitions changes, then do so. // Otherwise, just rebalance the data from any Partitions that were removed, if any. if (partitioner.isRebalanceOnClusterResize()) { @@ -669,6 +702,7 @@ public void setNodeIdentifiers(final Set updatedNodeIdentifiers, } protected void rebalance(final QueuePartition partition) { + logger.debug("Rebalancing Partition {}", partition); final FlowFileQueueContents contents = partition.packageForRebalance(rebalancingPartition.getSwapPartitionName()); rebalancingPartition.rebalance(contents); } @@ -988,26 +1022,15 @@ public void onNodeAdded(final NodeIdentifier nodeId) { } } - @Override - public void onNodeOffloaded(final NodeIdentifier nodeId) { - partitionWriteLock.lock(); - try { - final Set updatedNodeIds = new HashSet<>(nodeIdentifiers); - updatedNodeIds.remove(nodeId); - - logger.debug("Node Identifier {} offloaded. Node ID's changing from {} to {}", nodeId, nodeIdentifiers, updatedNodeIds); - setNodeIdentifiers(updatedNodeIds, false); - } finally { - partitionWriteLock.unlock(); - } - } - @Override public void onNodeRemoved(final NodeIdentifier nodeId) { partitionWriteLock.lock(); try { final Set updatedNodeIds = new HashSet<>(nodeIdentifiers); - updatedNodeIds.remove(nodeId); + final boolean removed = updatedNodeIds.remove(nodeId); + if (!removed) { + return; + } logger.debug("Node Identifier {} removed from cluster. Node ID's changing from {} to {}", nodeId, nodeIdentifiers, updatedNodeIds); setNodeIdentifiers(updatedNodeIds, false); @@ -1053,6 +1076,27 @@ public void onLocalNodeIdentifierSet(final NodeIdentifier localNodeId) { partitionWriteLock.unlock(); } } + + @Override + public void onNodeStateChange(final NodeIdentifier nodeId, final NodeConnectionState newState) { + partitionWriteLock.lock(); + try { + if (!offloaded) { + return; + } + + switch (newState) { + case OFFLOADED: + case OFFLOADING: + case DISCONNECTED: + case DISCONNECTING: + onNodeRemoved(nodeId); + break; + } + } finally { + partitionWriteLock.unlock(); + } + } } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/partition/NonLocalPartitionPartitioner.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/partition/NonLocalPartitionPartitioner.java index cffaefd5cbba..0953ce2c4dbd 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/partition/NonLocalPartitionPartitioner.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/partition/NonLocalPartitionPartitioner.java @@ -47,7 +47,7 @@ public QueuePartition getPartition(final FlowFileRecord flowFile, final QueuePar @Override public boolean isRebalanceOnClusterResize() { - return false; + return true; } From 25e448b1ecc05905dd3be6575082bd0b0faca5bb Mon Sep 17 00:00:00 2001 From: Jeff Storck Date: Tue, 25 Sep 2018 15:17:19 -0400 Subject: [PATCH 3/5] NIFI-5585 A node that was previously offloaded can now be reconnected to the cluster and queue flowfiles again Added Spock test for NonLocalPartitionPartitioner Updated NOTICE files for FontAwesome with the updated version (4.7.0) and URL to the free license Updated package-lock.json with the updated version of FontAwesome (4.7.0) Added method to FlowFileQueue interface to reset an offloaded queue Queues that are now immediately have the offloaded status reset once offloading finishes SocketLoadBalancedFlowFileQueue now ignores back-pressure when offloading flowfiles Cleaned up javascript in nf-cluster-table.js when creating markup for the node operation icons Fixed incorrect handling of a heartbeat from an offloaded node. Heartbeats from offloading or offloaded nodes will now be reported as an event, the heartbeat will be removed and ignored. Added unit tests and integration tests to cover offloading nodes Updated Cluster integration test class with accessor for the current cluster coordinator Updated Node integration test class's custom NiFiProperties implementation to return the load balancing port and a method to assert an offloaded node Added exclusion to top-level pom for ITSpec.class --- nifi-assembly/NOTICE | 2 +- .../nifi/controller/queue/FlowFileQueue.java | 14 ++ .../src/main/resources/META-INF/NOTICE | 2 +- .../coordination/ClusterCoordinator.java | 9 +- .../heartbeat/AbstractHeartbeatMonitor.java | 12 +- .../ThreadPoolRequestReplicator.java | 1 + .../node/NodeClusterCoordinator.java | 3 +- .../node/NodeClusterCoordinatorSpec.groovy | 99 ++++++++++++++ .../integration/OffloadNodeITSpec.groovy | 50 +++++++ .../nifi/cluster/integration/Cluster.java | 4 + .../apache/nifi/cluster/integration/Node.java | 15 ++ .../nifi/controller/StandardFlowService.java | 6 +- .../queue/StandardFlowFileQueue.java | 4 + .../SocketLoadBalancedFlowFileQueue.java | 72 +++++++--- .../controller/StandardFlowServiceSpec.groovy | 128 ++++++++++++++++++ .../NonLocalPartitionPartitionerSpec.groovy | 107 +++++++++++++++ .../TestWriteAheadFlowFileRepository.java | 4 + .../nifi/web/StandardNiFiServiceFacade.java | 3 +- .../src/main/frontend/package-lock.json | 6 +- .../src/main/resources/META-INF/NOTICE | 2 +- .../webapp/js/nf/cluster/nf-cluster-table.js | 41 +++--- .../command/nifi/nodes/DisconnectNode.java | 2 +- pom.xml | 3 + 23 files changed, 527 insertions(+), 62 deletions(-) create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinatorSpec.groovy create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/integration/OffloadNodeITSpec.groovy create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/StandardFlowServiceSpec.groovy create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/queue/clustered/partition/NonLocalPartitionPartitionerSpec.groovy diff --git a/nifi-assembly/NOTICE b/nifi-assembly/NOTICE index 0bea06a1047d..dd2bf6d647bd 100644 --- a/nifi-assembly/NOTICE +++ b/nifi-assembly/NOTICE @@ -1873,4 +1873,4 @@ SIL OFL 1.1 ****************** The following binary components are provided under the SIL Open Font License 1.1 - (SIL OFL 1.1) FontAwesome (4.6.1 - http://fortawesome.github.io/Font-Awesome/license/) + (SIL OFL 1.1) FontAwesome (4.7.0 - https://fontawesome.com/license/free) diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/queue/FlowFileQueue.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/queue/FlowFileQueue.java index 7cd0e3039347..8870f1d1ce18 100644 --- a/nifi-framework-api/src/main/java/org/apache/nifi/controller/queue/FlowFileQueue.java +++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/queue/FlowFileQueue.java @@ -267,8 +267,22 @@ public interface FlowFileQueue { void setLoadBalanceStrategy(LoadBalanceStrategy strategy, String partitioningAttribute); + /** + * Offloads the flowfiles in the queue to other nodes. This disables the queue from partition flowfiles locally. + *

+ * This operation is a no-op if the node that contains this queue is not in a cluster. + */ void offloadQueue(); + /** + * Resets a queue that has previously been offloaded. This allows the queue to partition flowfiles locally, and + * has no other effect on processors or remote process groups. + *

+ * This operation is a no-op if the queue is not currently offloaded or the node that contains this queue is not + * clustered. + */ + void resetOffloadedQueue(); + LoadBalanceStrategy getLoadBalanceStrategy(); void setLoadBalanceCompression(LoadBalanceCompression compression); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework-nar/src/main/resources/META-INF/NOTICE index e6a732228697..fae5c91e4a2e 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework-nar/src/main/resources/META-INF/NOTICE +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework-nar/src/main/resources/META-INF/NOTICE @@ -212,6 +212,6 @@ SIL OFL 1.1 ****************** The following binary components are provided under the SIL Open Font License 1.1 - (SIL OFL 1.1) FontAwesome (4.6.1 - http://fortawesome.github.io/Font-Awesome/license/) + (SIL OFL 1.1) FontAwesome (4.7.0 - https://fontawesome.com/license/free) diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/ClusterCoordinator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/ClusterCoordinator.java index 2ad0e709718b..fad51d1c4ad8 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/ClusterCoordinator.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/ClusterCoordinator.java @@ -72,7 +72,14 @@ public interface ClusterCoordinator { /** * Sends a request to the node to be offloaded. * The node will be marked as offloading immediately. - * + *

+ * When a node is offloaded: + *

    + *
  • all processors on the node are stopped
  • + *
  • all processors on the node are terminated
  • + *
  • all remote process groups on the node stop transmitting
  • + *
  • all flowfiles on the node are sent to other nodes in the cluster
  • + *
* @param nodeId the identifier of the node * @param offloadCode the code that represents why this node is being asked to be offloaded * @param explanation an explanation as to why the node is being asked to be offloaded 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 f6d09ab32e95..5fbe3f8bfd3d 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 @@ -228,12 +228,14 @@ private void processHeartbeat(final NodeHeartbeat heartbeat) { return; } - if (NodeConnectionState.OFFLOADED == connectionState) { - // Cluster Coordinator believes that node is offloaded, but let the node reconnect - clusterCoordinator.reportEvent(nodeId, Severity.INFO, "Received heartbeat from node that is offloaded. " + - "Marking as Disconnected and requesting that Node reconnect to cluster"); - clusterCoordinator.requestNodeConnect(nodeId, null); + if (NodeConnectionState.OFFLOADED == connectionState || NodeConnectionState.OFFLOADING == connectionState) { + // Cluster Coordinator can ignore this heartbeat since the node is offloaded + clusterCoordinator.reportEvent(nodeId, Severity.INFO, "Received heartbeat from node that is offloading " + + "or offloaded. Removing this heartbeat. Offloaded nodes will only be reconnected to the cluster by an " + + "explicit connection request or restarting the node."); + removeHeartbeat(nodeId); } + 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 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 0b2f1fe4ecea..b3a3ab965cf2 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 @@ -31,6 +31,7 @@ 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.OffloadedNodeMutableRequestException; import org.apache.nifi.cluster.manager.exception.UnknownNodeException; import org.apache.nifi.cluster.manager.exception.UriConstructionException; import org.apache.nifi.cluster.protocol.NodeIdentifier; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java index d697df122396..90eca2a38287 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java @@ -486,11 +486,12 @@ public void finishNodeOffload(final NodeIdentifier nodeId) { } if (state != NodeConnectionState.OFFLOADING) { - logger.warn("Attempted to finish node offload for {} but node is not in a offload state, it is currently {}.", nodeId, state); + logger.warn("Attempted to finish node offload for {} but node is not in the offloading state, it is currently {}.", nodeId, state); return; } logger.info("{} is now offloaded", nodeId); + updateNodeStatus(new NodeConnectionStatus(nodeId, NodeConnectionState.OFFLOADED)); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinatorSpec.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinatorSpec.groovy new file mode 100644 index 000000000000..2751b3ead18e --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinatorSpec.groovy @@ -0,0 +1,99 @@ +/* + * 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.nifi.cluster.coordination.flow.FlowElection +import org.apache.nifi.cluster.firewall.ClusterNodeFirewall +import org.apache.nifi.cluster.protocol.NodeIdentifier +import org.apache.nifi.cluster.protocol.NodeProtocolSender +import org.apache.nifi.cluster.protocol.impl.ClusterCoordinationProtocolSenderListener +import org.apache.nifi.cluster.protocol.message.OffloadMessage +import org.apache.nifi.components.state.Scope +import org.apache.nifi.components.state.StateManager +import org.apache.nifi.components.state.StateManagerProvider +import org.apache.nifi.controller.leader.election.LeaderElectionManager +import org.apache.nifi.events.EventReporter +import org.apache.nifi.reporting.Severity +import org.apache.nifi.state.MockStateMap +import org.apache.nifi.util.NiFiProperties +import org.apache.nifi.web.revision.RevisionManager +import spock.lang.Specification +import spock.util.concurrent.BlockingVariable + +import java.util.concurrent.TimeUnit + +class NodeClusterCoordinatorSpec extends Specification { + def "requestNodeOffload"() { + given: 'mocked collaborators' + def clusterCoordinationProtocolSenderListener = Mock(ClusterCoordinationProtocolSenderListener) + def eventReporter = Mock EventReporter + def stateManager = Mock StateManager + def stateMap = new MockStateMap([:], 1) + stateManager.getState(_ as Scope) >> stateMap + def stateManagerProvider = Mock StateManagerProvider + stateManagerProvider.getStateManager(_ as String) >> stateManager + + and: 'a NodeClusterCoordinator that manages node status in a synchronized list' + List nodeStatuses = [].asSynchronized() + def clusterCoordinator = new NodeClusterCoordinator(clusterCoordinationProtocolSenderListener, eventReporter, Mock(LeaderElectionManager), + Mock(FlowElection), Mock(ClusterNodeFirewall), + Mock(RevisionManager), NiFiProperties.createBasicNiFiProperties('src/test/resources/conf/nifi.properties', [:]), + Mock(NodeProtocolSender), stateManagerProvider) { + @Override + void notifyOthersOfNodeStatusChange(NodeConnectionStatus updatedStatus, boolean notifyAllNodes, boolean waitForCoordinator) { + nodeStatuses.add(updatedStatus) + } + } + + and: 'two nodes' + def nodeIdentifier1 = createNodeIdentifier 1 + def nodeIdentifier2 = createNodeIdentifier 2 + + and: 'node 1 is connected, node 2 is disconnected' + clusterCoordinator.updateNodeStatus new NodeConnectionStatus(nodeIdentifier1, NodeConnectionState.CONNECTED) + clusterCoordinator.updateNodeStatus new NodeConnectionStatus(nodeIdentifier2, NodeConnectionState.DISCONNECTED) + while (nodeStatuses.size() < 2) { + Thread.sleep(10) + } + nodeStatuses.clear() + + def waitForReportEvent = new BlockingVariable(5, TimeUnit.SECONDS) + + when: 'a node is requested to offload' + clusterCoordinator.requestNodeOffload nodeIdentifier2, OffloadCode.OFFLOADED, 'unit test for offloading node' + waitForReportEvent.get() + + then: 'no exceptions are thrown' + noExceptionThrown() + + and: 'expected methods on collaborators are invoked' + 1 * clusterCoordinationProtocolSenderListener.offload({ OffloadMessage msg -> msg.nodeId == nodeIdentifier2 } as OffloadMessage) + 1 * eventReporter.reportEvent(Severity.INFO, 'Clustering', { msg -> msg.contains "$nodeIdentifier2.apiAddress:$nodeIdentifier2.apiPort" } as String) >> { + waitForReportEvent.set(it) + } + + and: 'the status of the offloaded node is known by the cluster coordinator to be offloading' + nodeStatuses[0].nodeIdentifier == nodeIdentifier2 + nodeStatuses[0].state == NodeConnectionState.OFFLOADING + } + + private static NodeIdentifier createNodeIdentifier(final int index) { + new NodeIdentifier("node-id-$index", "localhost", 8000 + index, "localhost", 9000 + index, + "localhost", 10000 + index, 11000 + index, false) + } + +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/integration/OffloadNodeITSpec.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/integration/OffloadNodeITSpec.groovy new file mode 100644 index 000000000000..a8dd15835150 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/integration/OffloadNodeITSpec.groovy @@ -0,0 +1,50 @@ +/* + * 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.integration + +import org.apache.nifi.cluster.coordination.node.DisconnectionCode +import org.apache.nifi.cluster.coordination.node.OffloadCode +import spock.lang.Specification + +import java.util.concurrent.TimeUnit + +class OffloadNodeITSpec extends Specification { + def "requestNodeOffload"() { + given: 'a cluster with 3 nodes' + System.setProperty 'nifi.properties.file.path', 'src/test/resources/conf/nifi.properties' + def cluster = new Cluster() + cluster.start() + cluster.createNode() + def nodeToOffload = cluster.createNode() + cluster.createNode() + cluster.waitUntilAllNodesConnected 20, TimeUnit.SECONDS + + when: 'the node to offload is disconnected successfully' + cluster.currentClusterCoordinator.clusterCoordinator.requestNodeDisconnect nodeToOffload.identifier, DisconnectionCode.USER_DISCONNECTED, + 'integration test user disconnect' + cluster.currentClusterCoordinator.assertNodeDisconnects nodeToOffload.identifier, 10, TimeUnit.SECONDS + + and: 'the node to offload is requested to offload' + nodeToOffload.getClusterCoordinator().requestNodeOffload nodeToOffload.identifier, OffloadCode.OFFLOADED, 'integration test offload' + + then: 'the node has been successfully offloaded' + cluster.currentClusterCoordinator.assertNodeIsOffloaded nodeToOffload.identifier, 10, TimeUnit.SECONDS + + cleanup: + cluster.stop() + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java index dab073da85d3..370d6dc5a1c3 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java @@ -144,6 +144,10 @@ public Node createNode() { return node; } + public Node getCurrentClusterCoordinator() { + return getNodes().stream().filter(node -> node.hasRole(ClusterRoles.CLUSTER_COORDINATOR)).findFirst().orElse(null); + } + public Node waitForClusterCoordinator(final long time, final TimeUnit timeUnit) { return ClusterUtils.waitUntilNonNull(time, timeUnit, () -> getNodes().stream().filter(node -> node.hasRole(ClusterRoles.CLUSTER_COORDINATOR)).findFirst().orElse(null)); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Node.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Node.java index 3133736fbae0..b2a499aaba4e 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Node.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Node.java @@ -108,6 +108,8 @@ public String getProperty(String key) { return String.valueOf(nodeId.getSocketPort()); }else if(key.equals(NiFiProperties.WEB_HTTP_PORT)){ return String.valueOf(nodeId.getApiPort()); + }else if(key.equals(NiFiProperties.LOAD_BALANCE_PORT)){ + return String.valueOf(nodeId.getLoadBalancePort()); }else { return properties.getProperty(key); } @@ -386,4 +388,17 @@ public void assertNodeDisconnects(final NodeIdentifier nodeId, final long time, public void assertNodeIsConnected(final NodeIdentifier nodeId) { Assert.assertEquals(NodeConnectionState.CONNECTED, getClusterCoordinator().getConnectionStatus(nodeId).getState()); } + + /** + * Assert that the node with the given ID is offloaded (according to this node!) within the given amount of time + * + * @param nodeId id of the node + * @param time how long to wait + * @param timeUnit unit of time provided by the 'time' argument + */ + public void assertNodeIsOffloaded(final NodeIdentifier nodeId, final long time, final TimeUnit timeUnit) { + ClusterUtils.waitUntilConditionMet(time, timeUnit, + () -> getClusterCoordinator().getConnectionStatus(nodeId).getState() == NodeConnectionState.OFFLOADED, + () -> "Connection Status is " + getClusterCoordinator().getConnectionStatus(nodeId).toString()); + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java index 297595f4dfee..957357b112eb 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java @@ -691,20 +691,19 @@ private void handleOffloadRequest(final OffloadMessage request) throws Interrupt private void offload(final String explanation) throws InterruptedException { writeLock.lock(); try { - logger.info("Offloading node due to " + explanation); // mark node as offloading controller.setConnectionStatus(new NodeConnectionStatus(nodeId, NodeConnectionState.OFFLOADING, OffloadCode.OFFLOADED, explanation)); // request to stop all processors on node controller.stopAllProcessors(); - // request to stop all remote process groups - controller.getRootGroup().findAllRemoteProcessGroups().forEach(RemoteProcessGroup::stopTransmitting); // terminate all processors controller.getRootGroup().findAllProcessors() // filter stream, only stopped processors can be terminated .stream().filter(pn -> pn.getScheduledState() == ScheduledState.STOPPED) .forEach(pn -> pn.getProcessGroup().terminateProcessor(pn)); + // request to stop all remote process groups + controller.getRootGroup().findAllRemoteProcessGroups().forEach(RemoteProcessGroup::stopTransmitting); // offload all queues on node controller.getAllQueues().forEach(FlowFileQueue::offloadQueue); // wait for rebalance of flowfiles on all queues @@ -713,6 +712,7 @@ private void offload(final String explanation) throws InterruptedException { Thread.sleep(1000); } // finish offload + controller.getAllQueues().forEach(FlowFileQueue::resetOffloadedQueue); controller.setConnectionStatus(new NodeConnectionStatus(nodeId, NodeConnectionState.OFFLOADED, OffloadCode.OFFLOADED, explanation)); clusterCoordinator.finishNodeOffload(getNodeId()); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/StandardFlowFileQueue.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/StandardFlowFileQueue.java index ee222f4f2d89..8872ba7e6119 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/StandardFlowFileQueue.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/StandardFlowFileQueue.java @@ -78,6 +78,10 @@ public void stopLoadBalancing() { public void offloadQueue() { } + @Override + public void resetOffloadedQueue() { + } + @Override public boolean isActivelyLoadBalancing() { return false; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/SocketLoadBalancedFlowFileQueue.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/SocketLoadBalancedFlowFileQueue.java index 8ee072060bc7..66eaa3e99639 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/SocketLoadBalancedFlowFileQueue.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/SocketLoadBalancedFlowFileQueue.java @@ -189,34 +189,41 @@ public synchronized void setLoadBalanceStrategy(final LoadBalanceStrategy strate if (!offloaded) { // We are already load balancing but are changing how we are load balancing. final FlowFilePartitioner partitioner; - switch (strategy) { - case DO_NOT_LOAD_BALANCE: - partitioner = new LocalPartitionPartitioner(); - break; - case PARTITION_BY_ATTRIBUTE: - partitioner = new CorrelationAttributePartitioner(partitioningAttribute); - break; - case ROUND_ROBIN: - partitioner = new RoundRobinPartitioner(); - break; - case SINGLE_NODE: - partitioner = new FirstNodePartitioner(); - break; - default: - throw new IllegalArgumentException(); - } + partitioner = getPartitionerForLoadBalancingStrategy(strategy, partitioningAttribute); setFlowFilePartitioner(partitioner); } } + private FlowFilePartitioner getPartitionerForLoadBalancingStrategy(LoadBalanceStrategy strategy, String partitioningAttribute) { + FlowFilePartitioner partitioner; + switch (strategy) { + case DO_NOT_LOAD_BALANCE: + partitioner = new LocalPartitionPartitioner(); + break; + case PARTITION_BY_ATTRIBUTE: + partitioner = new CorrelationAttributePartitioner(partitioningAttribute); + break; + case ROUND_ROBIN: + partitioner = new RoundRobinPartitioner(); + break; + case SINGLE_NODE: + partitioner = new FirstNodePartitioner(); + break; + default: + throw new IllegalArgumentException(); + } + return partitioner; + } + @Override public void offloadQueue() { if (clusterCoordinator == null) { - // Not clustered, so don't change partitions + // Not clustered, cannot offload the queue to other nodes return; } + logger.debug("Setting queue {} on node {} as offloaded", this, clusterCoordinator.getLocalNodeIdentifier()); offloaded = true; partitionWriteLock.lock(); @@ -248,6 +255,26 @@ public void offloadQueue() { } } + @Override + public void resetOffloadedQueue() { + if (clusterCoordinator == null) { + // Not clustered, was not offloading the queue to other nodes + return; + } + + if (offloaded) { + // queue was offloaded previously, allow files to be added to the local partition + offloaded = false; + logger.debug("Queue {} on node {} was previously offloaded, resetting offloaded status to {}", + this, clusterCoordinator.getLocalNodeIdentifier(), offloaded); + // reset the partitioner based on the load balancing strategy, since offloading previously changed the partitioner + FlowFilePartitioner partitioner = getPartitionerForLoadBalancingStrategy(getLoadBalanceStrategy(), getPartitioningAttribute()); + setFlowFilePartitioner(partitioner); + logger.debug("Queue {} is no longer offloaded, restored load balance strategy to {} and partitioning attribute to \"{}\"", + this, getLoadBalanceStrategy(), getPartitioningAttribute()); + } + } + public synchronized void startLoadBalancing() { logger.debug("{} started. Will begin distributing FlowFiles across the cluster", this); @@ -884,8 +911,9 @@ public FlowFileRecord getFlowFile(final String flowFileUuid) throws IOException @Override public boolean isPropagateBackpressureAcrossNodes() { - // TODO: We will want to modify this when we have the ability to offload flowfiles from a node. - return true; + // If offloaded = false, the queue is not offloading; return true to honor backpressure + // If offloaded = true, the queue is offloading or has finished offloading; return false to ignore backpressure + return !offloaded; } @Override @@ -1086,6 +1114,12 @@ public void onNodeStateChange(final NodeIdentifier nodeId, final NodeConnectionS } switch (newState) { + case CONNECTED: + if (nodeId != null && nodeId.equals(clusterCoordinator.getLocalNodeIdentifier())) { + // the node with this queue was connected to the cluster, make sure the queue is not offloaded + resetOffloadedQueue(); + } + break; case OFFLOADED: case OFFLOADING: case DISCONNECTED: diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/StandardFlowServiceSpec.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/StandardFlowServiceSpec.groovy new file mode 100644 index 000000000000..63fedab8d4f9 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/StandardFlowServiceSpec.groovy @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.controller + +import org.apache.nifi.authorization.Authorizer +import org.apache.nifi.cluster.coordination.ClusterCoordinator +import org.apache.nifi.cluster.coordination.node.NodeConnectionState +import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus +import org.apache.nifi.cluster.coordination.node.OffloadCode +import org.apache.nifi.cluster.protocol.NodeIdentifier +import org.apache.nifi.cluster.protocol.impl.NodeProtocolSenderListener +import org.apache.nifi.cluster.protocol.message.OffloadMessage +import org.apache.nifi.components.state.Scope +import org.apache.nifi.components.state.StateManager +import org.apache.nifi.components.state.StateManagerProvider +import org.apache.nifi.controller.queue.FlowFileQueue +import org.apache.nifi.controller.status.ProcessGroupStatus +import org.apache.nifi.encrypt.StringEncryptor +import org.apache.nifi.groups.ProcessGroup +import org.apache.nifi.groups.RemoteProcessGroup +import org.apache.nifi.state.MockStateMap +import org.apache.nifi.util.NiFiProperties +import org.apache.nifi.web.revision.RevisionManager +import spock.lang.Specification +import spock.util.concurrent.BlockingVariable + +import java.util.concurrent.TimeUnit + +class StandardFlowServiceSpec extends Specification { + def "handle an OffloadMessage"() { + given: 'a node to offload' + def nodeToOffload = createNodeIdentifier 1 + + and: 'a simple flow with one root group and a single processor' + def stateManager = Mock StateManager + def stateMap = new MockStateMap([:], 1) + stateManager.getState(_ as Scope) >> stateMap + def stateManagerProvider = Mock StateManagerProvider + stateManagerProvider.getStateManager(_ as String) >> stateManager + + def rootGroup = Mock ProcessGroup + def flowController = Mock FlowController + flowController.getStateManagerProvider() >> stateManagerProvider + _ * flowController.rootGroup >> rootGroup + + def clusterCoordinator = Mock ClusterCoordinator + + def processGroupStatus = Mock ProcessGroupStatus + def processorNode = Mock ProcessorNode + def remoteProcessGroup = Mock RemoteProcessGroup + def flowFileQueue = Mock FlowFileQueue + + and: 'a flow service to handle the OffloadMessage' + def flowService = StandardFlowService.createClusteredInstance(flowController, NiFiProperties.createBasicNiFiProperties('src/test/resources/conf/nifi.properties', + [(NiFiProperties.CLUSTER_NODE_PROTOCOL_PORT): nodeToOffload.socketPort as String, + (NiFiProperties.WEB_HTTP_PORT) : nodeToOffload.apiPort as String, + (NiFiProperties.LOAD_BALANCE_PORT) : nodeToOffload.loadBalancePort as String]), + Mock(NodeProtocolSenderListener), clusterCoordinator, Mock(StringEncryptor), Mock(RevisionManager), Mock(Authorizer)) + + def waitForFinishOffload = new BlockingVariable(5, TimeUnit.SECONDS)//new CountDownLatch(1) + + when: 'the flow services receives an OffloadMessage' + flowService.handle(new OffloadMessage(nodeId: nodeToOffload, explanation: 'unit test offload'), [] as Set) + waitForFinishOffload.get() + + then: 'no exceptions are thrown' + noExceptionThrown() + + and: 'the connection status for the node in the flow controller is set to OFFLOADING' + 1 * flowController.setConnectionStatus({ NodeConnectionStatus status -> + status.nodeIdentifier.logicallyEquals(nodeToOffload) && status.state == NodeConnectionState.OFFLOADING && status.offloadCode == OffloadCode.OFFLOADED + } as NodeConnectionStatus) + + then: 'all processors are requested to stop' + 1 * flowController.stopAllProcessors() + + then: 'all processors are requested to terminate' + 1 * processorNode.scheduledState >> ScheduledState.STOPPED + 1 * processorNode.processGroup >> rootGroup + 1 * rootGroup.terminateProcessor({ ProcessorNode pn -> pn == processorNode } as ProcessorNode) + 1 * rootGroup.findAllProcessors() >> [processorNode] + + then: 'all remote process groups are requested to terminate' + 1 * remoteProcessGroup.stopTransmitting() + 1 * rootGroup.findAllRemoteProcessGroups() >> [remoteProcessGroup] + + then: 'all queues are requested to offload' + 1 * flowFileQueue.offloadQueue() + 1 * flowController.getAllQueues() >> [flowFileQueue] + + then: 'the queued count in the flow controller status is 0 to allow the offloading code to to complete' + 1 * flowController.getControllerStatus() >> processGroupStatus + 1 * processGroupStatus.getQueuedCount() >> 0 + + then: 'all queues are requested to reset to the original partitioner for the load balancing strategy' + 1 * flowFileQueue.resetOffloadedQueue() + 1 * flowController.getAllQueues() >> [flowFileQueue] + + then: 'the connection status for the node in the flow controller is set to OFFLOADED' + 1 * flowController.setConnectionStatus({ NodeConnectionStatus status -> + status.nodeIdentifier.logicallyEquals(nodeToOffload) && status.state == NodeConnectionState.OFFLOADED && status.offloadCode == OffloadCode.OFFLOADED + } as NodeConnectionStatus) + + then: 'the cluster coordinator is requested to finish the node offload' + 1 * clusterCoordinator.finishNodeOffload({ NodeIdentifier nodeIdentifier -> + nodeIdentifier.logicallyEquals(nodeToOffload) + } as NodeIdentifier) >> { waitForFinishOffload.set(it) } + } + + private static NodeIdentifier createNodeIdentifier(final int index) { + new NodeIdentifier("node-id-$index", "localhost", 8000 + index, "localhost", 9000 + index, + "localhost", 10000 + index, 11000 + index, false) + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/queue/clustered/partition/NonLocalPartitionPartitionerSpec.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/queue/clustered/partition/NonLocalPartitionPartitionerSpec.groovy new file mode 100644 index 000000000000..f3cfd4c47476 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/queue/clustered/partition/NonLocalPartitionPartitionerSpec.groovy @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.controller.queue.clustered.partition + + +import org.apache.nifi.controller.repository.FlowFileRecord +import spock.lang.Specification +import spock.lang.Unroll + +import java.util.concurrent.ConcurrentLinkedQueue +import java.util.concurrent.Executors +import java.util.concurrent.TimeUnit + +class NonLocalPartitionPartitionerSpec extends Specification { + + def "getPartition chooses local partition with 1 partition and throws IllegalStateException"() { + given: "a local partitioner using a local partition" + def partitioner = new NonLocalPartitionPartitioner() + def localPartition = Mock QueuePartition + def partitions = [localPartition] as QueuePartition[] + def flowFileRecord = Mock FlowFileRecord + + when: "a partition is requested from the partitioner" + partitioner.getPartition flowFileRecord, partitions, localPartition + + then: "an IllegalStateExceptions thrown" + thrown(IllegalStateException) + } + + @Unroll + def "getPartition chooses non-local partition with #maxPartitions partitions, #threads threads, #iterations iterations"() { + given: "a local partitioner" + def partitioner = new NonLocalPartitionPartitioner() + def partitions = new QueuePartition[maxPartitions] + + and: "a local partition" + def localPartition = Mock QueuePartition + partitions[0] = localPartition + + and: "one or more multiple partitions" + for (int id = 1; id < maxPartitions; ++id) { + def partition = Mock QueuePartition + partitions[id] = partition + } + + and: "an array to hold the resulting chosen partitions and an executor service with one or more threads" + def flowFileRecord = Mock FlowFileRecord + def chosenPartitions = [] as ConcurrentLinkedQueue + def executorService = Executors.newFixedThreadPool threads + + when: "a partition is requested from the partitioner for a given flowfile record and the existing partitions" + iterations.times { + executorService.submit { + chosenPartitions.add partitioner.getPartition(flowFileRecord, partitions, localPartition) + } + } + executorService.shutdown() + try { + while (!executorService.awaitTermination(10, TimeUnit.MILLISECONDS)) { + Thread.sleep(10) + } + } catch (InterruptedException e) { + executorService.shutdownNow() + Thread.currentThread().interrupt() + } + + then: "no exceptions are thrown" + noExceptionThrown() + + and: "there is a chosen partition for each iteration" + chosenPartitions.size() == iterations + + and: "each chosen partition is a remote partition and is one of the existing partitions" + def validChosenPartitions = chosenPartitions.findAll { it != localPartition && partitions.contains(it) } + + and: "there is a valid chosen partition for each iteration" + validChosenPartitions.size() == iterations + + and: "there are no other mock interactions" + 0 * _ + + where: + maxPartitions | threads | iterations + 2 | 1 | 1 + 2 | 1 | 10 + 2 | 1 | 100 + 2 | 10 | 1000 + 5 | 1 | 1 + 5 | 1 | 10 + 5 | 1 | 100 + 5 | 10 | 1000 + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java index 878ad130d42c..a3ee5c16b3d8 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java @@ -113,6 +113,10 @@ public void stopLoadBalancing() { public void offloadQueue() { } + @Override + public void resetOffloadedQueue() { + } + @Override public boolean isActivelyLoadBalancing() { return false; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java index 4ef241e8f59c..d8751184ab63 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java @@ -4707,7 +4707,8 @@ public void deleteNode(final String nodeId) { final NodeConnectionStatus nodeConnectionStatus = clusterCoordinator.getConnectionStatus(nodeIdentifier); if (!nodeConnectionStatus.getState().equals(NodeConnectionState.OFFLOADED) && !nodeConnectionStatus.getState().equals(NodeConnectionState.DISCONNECTED)) { - throw new IllegalNodeDeletionException("Cannot remove Node with ID " + nodeId + " because it is not disconnected, current state = " + nodeConnectionStatus.getState()); + throw new IllegalNodeDeletionException("Cannot remove Node with ID " + nodeId + + " because it is not disconnected or offloaded, current state = " + nodeConnectionStatus.getState()); } clusterCoordinator.removeNode(nodeIdentifier, userDn); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/frontend/package-lock.json b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/frontend/package-lock.json index 1d848a61a1c1..3f505a2b9466 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/frontend/package-lock.json +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/frontend/package-lock.json @@ -362,9 +362,9 @@ "integrity": "sha512-ipiDYhdQSCZ4hSbX4rMW+XzNKMD1prg/sTvoVmSLkuQ1MVlwjJQQA+sW8tMYR3BLUr9KjodFV4pvzunvRhd33Q==" }, "font-awesome": { - "version": "4.6.1", - "resolved": "https://registry.npmjs.org/font-awesome/-/font-awesome-4.6.1.tgz", - "integrity": "sha1-VHJl+0xFu+2Qq4vE93qXs3uFKhI=" + "version": "4.7.0", + "resolved": "https://registry.npmjs.org/font-awesome/-/font-awesome-4.7.0.tgz", + "integrity": "sha1-j6jPBBGhoxr9B7BtKQK7n8gVoTM=" }, "has-color": { "version": "0.1.7", diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/META-INF/NOTICE index c0b74bc8b47d..300c6b9bf2a7 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/META-INF/NOTICE +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/META-INF/NOTICE @@ -8,4 +8,4 @@ SIL OFL 1.1 ****************** The following binary components are provided under the SIL Open Font License 1.1 - (SIL OFL 1.1) FontAwesome (4.6.1 - http://fortawesome.github.io/Font-Awesome/license/) + (SIL OFL 1.1) FontAwesome (4.7.0 - https://fontawesome.com/license/free) diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/cluster/nf-cluster-table.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/cluster/nf-cluster-table.js index 0dc74d7cad5a..f90cd3ab8c13 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/cluster/nf-cluster-table.js +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/cluster/nf-cluster-table.js @@ -630,34 +630,25 @@ // only allow the admin to modify the cluster if (nfCommon.canModifyController()) { var actionFormatter = function (row, cell, value, columnDef, dataContext) { - var canDisconnect = false; - var canConnect = false; - var isOffloaded = false; - - // determine the current status + var connectDiv = '
'; + var deleteDiv = '
'; + var disconnectDiv = '
'; + var offloadDiv = '
'; + var markup = ''; + + // determine the current status and create the appropriate markup if (dataContext.status === 'CONNECTED' || dataContext.status === 'CONNECTING') { - canDisconnect = true; - } - if (dataContext.status === 'DISCONNECTED') { - canConnect = true; - } - if (dataContext.status === 'OFFLOADED') { - isOffloaded = true; - } - - // return the appropriate markup - if (canConnect) { - return '
' + - '
' + - '
'; - } else if (canDisconnect) { - return '
'; - } else if (isOffloaded) { - return '
' + - '
'; + markup += disconnectDiv; + } else if (dataContext.status === 'DISCONNECTED') { + markup += connectDiv + offloadDiv + deleteDiv; + } else if (dataContext.status === 'OFFLOADED') { + markup += connectDiv + deleteDiv; } else { - return '
 
'; + markup += '
 
'; } + + return markup; }; columnModel.push({ diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/nodes/DisconnectNode.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/nodes/DisconnectNode.java index 98fa03a11541..65a7e72efd01 100644 --- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/nodes/DisconnectNode.java +++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/nodes/DisconnectNode.java @@ -57,7 +57,7 @@ public NodeResult doExecute(NiFiClient client, Properties properties) throws NiF NodeDTO nodeDto = new NodeDTO(); nodeDto.setNodeId(nodeId); - // TODO There's no constant for node status in + // TODO There are no constants for the DISCONNECT node status nodeDto.setStatus("DISCONNECTING"); NodeEntity nodeEntity = new NodeEntity(); nodeEntity.setNode(nodeDto); diff --git a/pom.xml b/pom.xml index 7d433b4247e7..b86cada10379 100644 --- a/pom.xml +++ b/pom.xml @@ -342,6 +342,9 @@ **/Test*.class **/*Spec.class + + **/*ITSpec.class + true -Xmx1G -Djava.net.preferIPv4Stack=true From b8f7591837e22658eec544b57572a5efcf3d6a57 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Mon, 8 Oct 2018 09:53:14 -0400 Subject: [PATCH 4/5] NIFI-5585: Adjustments to the Connection Load Balancing to ensure that node offloading works smoothly Signed-off-by: Jeff Storck --- .../client/async/AsyncLoadBalanceClient.java | 2 ++ .../async/nio/NioAsyncLoadBalanceClient.java | 4 ++++ .../NioAsyncLoadBalanceClientRegistry.java | 20 +++++++++++++++---- .../nio/NioAsyncLoadBalanceClientTask.java | 8 ++------ 4 files changed, 24 insertions(+), 10 deletions(-) diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/client/async/AsyncLoadBalanceClient.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/client/async/AsyncLoadBalanceClient.java index 1bb405321f0b..8673a8b3eb49 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/client/async/AsyncLoadBalanceClient.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/client/async/AsyncLoadBalanceClient.java @@ -39,6 +39,8 @@ void register(String connectionId, BooleanSupplier emptySupplier, Supplier getRegisteredPartitions() { return new HashMap<>(registeredPartitions); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/client/async/nio/NioAsyncLoadBalanceClientRegistry.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/client/async/nio/NioAsyncLoadBalanceClientRegistry.java index 514a58c998f3..3322035db491 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/client/async/nio/NioAsyncLoadBalanceClientRegistry.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/client/async/nio/NioAsyncLoadBalanceClientRegistry.java @@ -67,15 +67,27 @@ public synchronized void register(final String connectionId, final NodeIdentifie @Override public synchronized void unregister(final String connectionId, final NodeIdentifier nodeId) { - final Set clients = clientMap.remove(nodeId); + final Set clients = clientMap.get(nodeId); if (clients == null) { return; } - clients.forEach(client -> client.unregister(connectionId)); + final Set toRemove = new HashSet<>(); + for (final AsyncLoadBalanceClient client : clients) { + client.unregister(connectionId); + if (client.getRegisteredConnectionCount() == 0) { + toRemove.add(client); + } + } + + clients.removeAll(toRemove); + allClients.removeAll(toRemove); + + if (clients.isEmpty()) { + clientMap.remove(nodeId); + } - allClients.removeAll(clients); - logger.debug("Un-registered Connection with ID {} so that it will no longer send data to Node {}", connectionId, nodeId); + logger.debug("Un-registered Connection with ID {} so that it will no longer send data to Node {}; {} clients were removed", connectionId, nodeId, toRemove.size()); } private Set registerClients(final NodeIdentifier nodeId) { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/client/async/nio/NioAsyncLoadBalanceClientTask.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/client/async/nio/NioAsyncLoadBalanceClientTask.java index 35ea5f9840d7..5c8073aa333d 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/client/async/nio/NioAsyncLoadBalanceClientTask.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/client/async/nio/NioAsyncLoadBalanceClientTask.java @@ -66,13 +66,9 @@ public void run() { } final NodeConnectionState connectionState = connectionStatus.getState(); - if (connectionState == NodeConnectionState.DISCONNECTED || connectionState == NodeConnectionState.DISCONNECTING) { - client.nodeDisconnected(); - continue; - } - if (connectionState != NodeConnectionState.CONNECTED) { - logger.debug("Client {} is for node that is not currently connected (state = {}) so will not communicate with node", client, connectionState); + logger.debug("Notifying Client {} that node is not connected because current state is {}", client, connectionState); + client.nodeDisconnected(); continue; } From 993baaeac5d04bbc7bce086e70cbc4e5c991a337 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Tue, 9 Oct 2018 14:54:21 -0400 Subject: [PATCH 5/5] NIFI-5585: Addressed bug in calculating swap size of a queue partition when rebalancing --- .../queue/SwappablePriorityQueue.java | 24 +++++++++++-------- 1 file changed, 14 insertions(+), 10 deletions(-) diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/SwappablePriorityQueue.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/SwappablePriorityQueue.java index 66b594db9ff9..6dfa77deea88 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/SwappablePriorityQueue.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/SwappablePriorityQueue.java @@ -878,7 +878,7 @@ protected void incrementActiveQueueSize(final int count, final long bytes) { original.getSwappedCount(), original.getSwappedBytes(), original.getSwapFileCount(), original.getUnacknowledgedCount(), original.getUnacknowledgedBytes()); - updated = size.compareAndSet(original, newSize); + updated = updateSize(original, newSize); if (updated) { logIfNegative(original, newSize, "active"); @@ -908,7 +908,8 @@ private void incrementUnacknowledgedQueueSize(final int count, final long bytes) final FlowFileQueueSize newSize = new FlowFileQueueSize(original.getActiveCount(), original.getActiveBytes(), original.getSwappedCount(), original.getSwappedBytes(), original.getSwapFileCount(), original.getUnacknowledgedCount() + count, original.getUnacknowledgedBytes() + bytes); - updated = size.compareAndSet(original, newSize); + + updated = updateSize(original, newSize); if (updated) { logIfNegative(original, newSize, "Unacknowledged"); @@ -949,7 +950,6 @@ public FlowFileQueueContents packageForRebalance(final String newPartitionName) writeLock.lock(); try { final List activeRecords = new ArrayList<>(this.activeQueue); - activeRecords.addAll(this.swapQueue); final List updatedSwapLocations = new ArrayList<>(swapLocations.size()); for (final String swapLocation : swapLocations) { @@ -963,23 +963,27 @@ public FlowFileQueueContents packageForRebalance(final String newPartitionName) this.swapLocations.clear(); this.activeQueue.clear(); - this.swapQueue.clear(); + + final int swapQueueCount = swapQueue.size(); + final long swapQueueBytes = swapQueue.stream().mapToLong(FlowFileRecord::getSize).sum(); + activeRecords.addAll(swapQueue); + swapQueue.clear(); this.swapMode = false; - QueueSize swapSize = new QueueSize(0, 0L); - boolean updated = false; - while (!updated) { + QueueSize swapSize; + boolean updated; + do { final FlowFileQueueSize currentSize = getFlowFileQueueSize(); - swapSize = new QueueSize(currentSize.getSwappedCount(), currentSize.getSwappedBytes()); + swapSize = new QueueSize(currentSize.getSwappedCount() - swapQueueCount, currentSize.getSwappedBytes() - swapQueueBytes); final FlowFileQueueSize updatedSize = new FlowFileQueueSize(0, 0, 0, 0, 0, currentSize.getUnacknowledgedCount(), currentSize.getUnacknowledgedBytes()); updated = updateSize(currentSize, updatedSize); - } + } while (!updated); return new FlowFileQueueContents(activeRecords, updatedSwapLocations, swapSize); } finally { - writeLock.unlock("transfer(SwappablePriorityQueue)"); + writeLock.unlock("packageForRebalance(SwappablePriorityQueue)"); } }