NIFI-1897:

- Refactoring to allow requests to be replicated from a node to other nodes
- Renaming cluster node connection/read timeout properties.
- Renaming NCM DN to Cluster Coordinator DN.
- Fixing default values in properties.
- Starting to fix Spring context to load correctly in standalone mode.
- Using the cluster protocol to handle connection failures instead of heartbeats.
- Ensured replicate call is returned from ControllerResource.
- Ensure the appropriate classloader when serializing templates.
- Handling when the flow contents are null.
- This closes #488
This commit is contained in:
Mark Payne 2016-05-19 10:42:39 -04:00 committed by Matt Gilman
parent cd011731ab
commit df0e4e7960
189 changed files with 5474 additions and 13029 deletions

View File

@ -27,6 +27,7 @@ import java.util.Objects;
* @Threadsafe
*/
public class Revision implements Serializable {
private static final long serialVersionUID = 988658790374170022L;
/**
* the version number

View File

@ -447,37 +447,18 @@ language governing permissions and limitations under the License. -->
<!-- nifi.properties: cluster common properties (cluster manager and nodes must have same values) -->
<nifi.cluster.protocol.heartbeat.interval>5 sec</nifi.cluster.protocol.heartbeat.interval>
<nifi.cluster.protocol.is.secure>false</nifi.cluster.protocol.is.secure>
<nifi.cluster.protocol.socket.timeout>30 sec</nifi.cluster.protocol.socket.timeout>
<nifi.cluster.protocol.connection.handshake.timeout>45 sec</nifi.cluster.protocol.connection.handshake.timeout>
<nifi.cluster.protocol.use.multicast>false</nifi.cluster.protocol.use.multicast>
<nifi.cluster.protocol.multicast.address />
<nifi.cluster.protocol.multicast.port />
<nifi.cluster.protocol.multicast.service.broadcast.delay>500 ms</nifi.cluster.protocol.multicast.service.broadcast.delay>
<nifi.cluster.protocol.multicast.service.locator.attempts>3</nifi.cluster.protocol.multicast.service.locator.attempts>
<nifi.cluster.protocol.multicast.service.locator.attempts.delay>1 sec</nifi.cluster.protocol.multicast.service.locator.attempts.delay>
<!-- nifi.properties: cluster node properties (only configure for cluster nodes) -->
<nifi.cluster.is.node>false</nifi.cluster.is.node>
<nifi.cluster.node.address />
<nifi.cluster.node.protocol.port />
<nifi.cluster.node.protocol.threads>2</nifi.cluster.node.protocol.threads>
<nifi.cluster.node.unicast.manager.address />
<nifi.cluster.node.unicast.manager.protocol.port />
<!-- nifi.properties: cluster manager properties (only configure for cluster manager) -->
<nifi.cluster.is.manager>false</nifi.cluster.is.manager>
<nifi.cluster.manager.address />
<nifi.cluster.manager.protocol.port />
<nifi.cluster.manager.node.firewall.file />
<nifi.cluster.manager.node.event.history.size>10</nifi.cluster.manager.node.event.history.size>
<nifi.cluster.manager.node.api.connection.timeout>30 sec</nifi.cluster.manager.node.api.connection.timeout>
<nifi.cluster.manager.node.api.read.timeout>30 sec</nifi.cluster.manager.node.api.read.timeout>
<nifi.cluster.manager.node.api.request.threads>10</nifi.cluster.manager.node.api.request.threads>
<nifi.cluster.manager.flow.retrieval.delay>5 sec</nifi.cluster.manager.flow.retrieval.delay>
<nifi.cluster.manager.protocol.threads>10</nifi.cluster.manager.protocol.threads>
<nifi.cluster.manager.safemode.duration>0 sec</nifi.cluster.manager.safemode.duration>
<nifi.cluster.request.replication.claim.timeout>1 min</nifi.cluster.request.replication.claim.timeout>
<nifi.cluster.node.protocol.threads>10</nifi.cluster.node.protocol.threads>
<nifi.cluster.node.event.history.size>25</nifi.cluster.node.event.history.size>
<nifi.cluster.node.connection.timeout>5 sec</nifi.cluster.node.connection.timeout>
<nifi.cluster.node.read.timeout>5 sec</nifi.cluster.node.read.timeout>
<nifi.cluster.firewall.file />
<nifi.cluster.request.replication.claim.timeout>15 secs</nifi.cluster.request.replication.claim.timeout>
<!-- nifi.properties: zookeeper properties -->
<nifi.zookeeper.connect.string></nifi.zookeeper.connect.string>

View File

@ -150,22 +150,16 @@ public class NiFiProperties extends Properties {
// cluster common properties
public static final String CLUSTER_PROTOCOL_HEARTBEAT_INTERVAL = "nifi.cluster.protocol.heartbeat.interval";
public static final String CLUSTER_PROTOCOL_IS_SECURE = "nifi.cluster.protocol.is.secure";
public static final String CLUSTER_PROTOCOL_SOCKET_TIMEOUT = "nifi.cluster.protocol.socket.timeout";
public static final String CLUSTER_PROTOCOL_CONNECTION_HANDSHAKE_TIMEOUT = "nifi.cluster.protocol.connection.handshake.timeout";
public static final String CLUSTER_PROTOCOL_USE_MULTICAST = "nifi.cluster.protocol.use.multicast";
public static final String CLUSTER_PROTOCOL_MULTICAST_ADDRESS = "nifi.cluster.protocol.multicast.address";
public static final String CLUSTER_PROTOCOL_MULTICAST_PORT = "nifi.cluster.protocol.multicast.port";
public static final String CLUSTER_PROTOCOL_MULTICAST_SERVICE_BROADCAST_DELAY = "nifi.cluster.protocol.multicast.service.broadcast.delay";
public static final String CLUSTER_PROTOCOL_MULTICAST_SERVICE_LOCATOR_ATTEMPTS = "nifi.cluster.protocol.multicast.service.locator.attempts";
public static final String CLUSTER_PROTOCOL_MULTICAST_SERVICE_LOCATOR_ATTEMPTS_DELAY = "nifi.cluster.protocol.multicast.service.locator.attempts.delay";
// cluster node properties
public static final String CLUSTER_IS_NODE = "nifi.cluster.is.node";
public static final String CLUSTER_NODE_ADDRESS = "nifi.cluster.node.address";
public static final String CLUSTER_NODE_PROTOCOL_PORT = "nifi.cluster.node.protocol.port";
public static final String CLUSTER_NODE_PROTOCOL_THREADS = "nifi.cluster.node.protocol.threads";
public static final String CLUSTER_NODE_UNICAST_MANAGER_ADDRESS = "nifi.cluster.node.unicast.manager.address";
public static final String CLUSTER_NODE_UNICAST_MANAGER_PROTOCOL_PORT = "nifi.cluster.node.unicast.manager.protocol.port";
public static final String REQUEST_REPLICATION_CLAIM_TIMEOUT = "nifi.cluster.request.replication.claim.timeout";
public static final String CLUSTER_NODE_CONNECTION_TIMEOUT = "nifi.cluster.node.connection.timeout";
public static final String CLUSTER_NODE_READ_TIMEOUT = "nifi.cluster.node.read.timeout";
public static final String CLUSTER_FIREWALL_FILE = "nifi.cluster.firewall.file";
// zookeeper properties
public static final String ZOOKEEPER_CONNECT_STRING = "nifi.zookeeper.connect.string";
@ -173,20 +167,6 @@ public class NiFiProperties extends Properties {
public static final String ZOOKEEPER_SESSION_TIMEOUT = "nifi.zookeeper.session.timeout";
public static final String ZOOKEEPER_ROOT_NODE = "nifi.zookeeper.root.node";
// cluster manager properties
public static final String CLUSTER_IS_MANAGER = "nifi.cluster.is.manager";
public static final String CLUSTER_MANAGER_ADDRESS = "nifi.cluster.manager.address";
public static final String CLUSTER_MANAGER_PROTOCOL_PORT = "nifi.cluster.manager.protocol.port";
public static final String CLUSTER_MANAGER_NODE_FIREWALL_FILE = "nifi.cluster.manager.node.firewall.file";
public static final String CLUSTER_MANAGER_NODE_EVENT_HISTORY_SIZE = "nifi.cluster.manager.node.event.history.size";
public static final String CLUSTER_MANAGER_NODE_API_CONNECTION_TIMEOUT = "nifi.cluster.manager.node.api.connection.timeout";
public static final String CLUSTER_MANAGER_NODE_API_READ_TIMEOUT = "nifi.cluster.manager.node.api.read.timeout";
public static final String CLUSTER_MANAGER_NODE_API_REQUEST_THREADS = "nifi.cluster.manager.node.api.request.threads";
public static final String CLUSTER_MANAGER_FLOW_RETRIEVAL_DELAY = "nifi.cluster.manager.flow.retrieval.delay";
public static final String CLUSTER_MANAGER_PROTOCOL_THREADS = "nifi.cluster.manager.protocol.threads";
public static final String CLUSTER_MANAGER_SAFEMODE_DURATION = "nifi.cluster.manager.safemode.duration";
public static final String REQUEST_REPLICATION_CLAIM_TIMEOUT = "nifi.cluster.request.replication.claim.timeout";
// kerberos properties
public static final String KERBEROS_KRB5_FILE = "nifi.kerberos.krb5.file";
@ -236,22 +216,12 @@ public class NiFiProperties extends Properties {
public static final String DEFAULT_CLUSTER_PROTOCOL_MULTICAST_SERVICE_BROADCAST_DELAY = "500 ms";
public static final int DEFAULT_CLUSTER_PROTOCOL_MULTICAST_SERVICE_LOCATOR_ATTEMPTS = 3;
public static final String DEFAULT_CLUSTER_PROTOCOL_MULTICAST_SERVICE_LOCATOR_ATTEMPTS_DELAY = "1 sec";
public static final String DEFAULT_CLUSTER_PROTOCOL_SOCKET_TIMEOUT = "30 sec";
public static final String DEFAULT_CLUSTER_PROTOCOL_CONNECTION_HANDSHAKE_TIMEOUT = "45 sec";
public static final String DEFAULT_CLUSTER_NODE_READ_TIMEOUT = "5 sec";
public static final String DEFAULT_CLUSTER_NODE_CONNECTION_TIMEOUT = "5 sec";
// cluster node defaults
public static final int DEFAULT_CLUSTER_NODE_PROTOCOL_THREADS = 2;
// cluster manager defaults
public static final int DEFAULT_CLUSTER_MANAGER_NODE_EVENT_HISTORY_SIZE = 10;
public static final String DEFAULT_CLUSTER_MANAGER_NODE_API_CONNECTION_TIMEOUT = "30 sec";
public static final String DEFAULT_CLUSTER_MANAGER_NODE_API_READ_TIMEOUT = "30 sec";
public static final int DEFAULT_CLUSTER_MANAGER_NODE_API_NUM_REQUEST_THREADS = 10;
public static final String DEFAULT_CLUSTER_MANAGER_FLOW_RETRIEVAL_DELAY = "5 sec";
public static final int DEFAULT_CLUSTER_MANAGER_PROTOCOL_THREADS = 10;
public static final String DEFAULT_CLUSTER_MANAGER_SAFEMODE_DURATION = "0 sec";
public static final String DEFAULT_REQUEST_REPLICATION_CLAIM_TIMEOUT = "1 min";
public static final String DEFAULT_REQUEST_REPLICATION_CLAIM_TIMEOUT = "15 secs";
// state management defaults
public static final String DEFAULT_STATE_MANAGEMENT_CONFIG_FILE = "conf/state-management.xml";
@ -643,31 +613,13 @@ public class NiFiProperties extends Properties {
return getClusterProtocolHeartbeatInterval();
}
public String getClusterProtocolSocketTimeout() {
return getProperty(CLUSTER_PROTOCOL_SOCKET_TIMEOUT, DEFAULT_CLUSTER_PROTOCOL_SOCKET_TIMEOUT);
public String getClusterNodeReadTimeout() {
return getProperty(CLUSTER_NODE_READ_TIMEOUT, DEFAULT_CLUSTER_NODE_READ_TIMEOUT);
}
public String getClusterProtocolConnectionHandshakeTimeout() {
return getProperty(CLUSTER_PROTOCOL_CONNECTION_HANDSHAKE_TIMEOUT,
DEFAULT_CLUSTER_PROTOCOL_CONNECTION_HANDSHAKE_TIMEOUT);
}
public boolean getClusterProtocolUseMulticast() {
return Boolean.parseBoolean(getProperty(CLUSTER_PROTOCOL_USE_MULTICAST));
}
public InetSocketAddress getClusterProtocolMulticastAddress() {
try {
String multicastAddress = getProperty(CLUSTER_PROTOCOL_MULTICAST_ADDRESS);
int multicastPort = Integer.parseInt(getProperty(CLUSTER_PROTOCOL_MULTICAST_PORT));
return new InetSocketAddress(multicastAddress, multicastPort);
} catch (Exception ex) {
throw new RuntimeException("Invalid multicast address/port due to: " + ex, ex);
}
}
public String getClusterProtocolMulticastServiceBroadcastDelay() {
return getProperty(CLUSTER_PROTOCOL_MULTICAST_SERVICE_BROADCAST_DELAY);
public String getClusterNodeConnectionTimeout() {
return getProperty(CLUSTER_NODE_CONNECTION_TIMEOUT,
DEFAULT_CLUSTER_NODE_CONNECTION_TIMEOUT);
}
public File getPersistentStateDirectory() {
@ -680,19 +632,6 @@ public class NiFiProperties extends Properties {
return file;
}
public int getClusterProtocolMulticastServiceLocatorAttempts() {
try {
return Integer
.parseInt(getProperty(CLUSTER_PROTOCOL_MULTICAST_SERVICE_LOCATOR_ATTEMPTS));
} catch (NumberFormatException nfe) {
return DEFAULT_CLUSTER_PROTOCOL_MULTICAST_SERVICE_LOCATOR_ATTEMPTS;
}
}
public String getClusterProtocolMulticastServiceLocatorAttemptsDelay() {
return getProperty(CLUSTER_PROTOCOL_MULTICAST_SERVICE_LOCATOR_ATTEMPTS_DELAY,
DEFAULT_CLUSTER_PROTOCOL_MULTICAST_SERVICE_LOCATOR_ATTEMPTS_DELAY);
}
// getters for cluster node properties //
public boolean isNode() {
@ -728,48 +667,13 @@ public class NiFiProperties extends Properties {
}
}
public InetSocketAddress getClusterNodeUnicastManagerProtocolAddress() {
try {
String socketAddress = getProperty(CLUSTER_NODE_UNICAST_MANAGER_ADDRESS);
if (StringUtils.isBlank(socketAddress)) {
socketAddress = "localhost";
}
int socketPort = Integer
.parseInt(getProperty(CLUSTER_NODE_UNICAST_MANAGER_PROTOCOL_PORT));
return InetSocketAddress.createUnresolved(socketAddress, socketPort);
} catch (Exception ex) {
throw new RuntimeException("Invalid unicast manager address/port due to: " + ex, ex);
}
public boolean isClustered() {
return Boolean.parseBoolean(getProperty(CLUSTER_IS_NODE));
}
// getters for cluster manager properties //
public boolean isClusterManager() {
return Boolean.parseBoolean(getProperty(CLUSTER_IS_MANAGER));
}
public InetSocketAddress getClusterManagerProtocolAddress() {
try {
String socketAddress = getProperty(CLUSTER_MANAGER_ADDRESS);
if (StringUtils.isBlank(socketAddress)) {
socketAddress = "localhost";
}
int socketPort = getClusterManagerProtocolPort();
return InetSocketAddress.createUnresolved(socketAddress, socketPort);
} catch (Exception ex) {
throw new RuntimeException("Invalid manager protocol address/port due to: " + ex, ex);
}
}
public Integer getClusterManagerProtocolPort() {
try {
return Integer.parseInt(getProperty(CLUSTER_MANAGER_PROTOCOL_PORT));
} catch (NumberFormatException nfe) {
return null;
}
}
public File getClusterManagerNodeFirewallFile() {
final String firewallFile = getProperty(CLUSTER_MANAGER_NODE_FIREWALL_FILE);
public File getClusterNodeFirewallFile() {
final String firewallFile = getProperty(CLUSTER_FIREWALL_FILE);
if (StringUtils.isBlank(firewallFile)) {
return null;
} else {
@ -777,50 +681,6 @@ public class NiFiProperties extends Properties {
}
}
public int getClusterManagerNodeEventHistorySize() {
try {
return Integer.parseInt(getProperty(CLUSTER_MANAGER_NODE_EVENT_HISTORY_SIZE));
} catch (NumberFormatException nfe) {
return DEFAULT_CLUSTER_MANAGER_NODE_EVENT_HISTORY_SIZE;
}
}
public String getClusterManagerNodeApiConnectionTimeout() {
return getProperty(CLUSTER_MANAGER_NODE_API_CONNECTION_TIMEOUT,
DEFAULT_CLUSTER_MANAGER_NODE_API_CONNECTION_TIMEOUT);
}
public String getClusterManagerNodeApiReadTimeout() {
return getProperty(CLUSTER_MANAGER_NODE_API_READ_TIMEOUT,
DEFAULT_CLUSTER_MANAGER_NODE_API_READ_TIMEOUT);
}
public int getClusterManagerNodeApiRequestThreads() {
try {
return Integer.parseInt(getProperty(CLUSTER_MANAGER_NODE_API_REQUEST_THREADS));
} catch (NumberFormatException nfe) {
return DEFAULT_CLUSTER_MANAGER_NODE_API_NUM_REQUEST_THREADS;
}
}
public String getClusterManagerFlowRetrievalDelay() {
return getProperty(CLUSTER_MANAGER_FLOW_RETRIEVAL_DELAY,
DEFAULT_CLUSTER_MANAGER_FLOW_RETRIEVAL_DELAY);
}
public int getClusterManagerProtocolThreads() {
try {
return Integer.parseInt(getProperty(CLUSTER_MANAGER_PROTOCOL_THREADS));
} catch (NumberFormatException nfe) {
return DEFAULT_CLUSTER_MANAGER_PROTOCOL_THREADS;
}
}
public String getClusterManagerSafeModeDuration() {
return getProperty(CLUSTER_MANAGER_SAFEMODE_DURATION,
DEFAULT_CLUSTER_MANAGER_SAFEMODE_DURATION);
}
public String getClusterProtocolManagerToNodeApiScheme() {
final String isSecureProperty = getProperty(CLUSTER_PROTOCOL_IS_SECURE);
if (Boolean.valueOf(isSecureProperty)) {

View File

@ -33,8 +33,6 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.nifi.logging.NiFiLog;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -45,7 +43,7 @@ import org.slf4j.LoggerFactory;
public abstract class SocketListener {
private static final int DEFAULT_SHUTDOWN_LISTENER_SECONDS = 5;
private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(SocketListener.class));
private static final Logger logger = LoggerFactory.getLogger(SocketListener.class);
private volatile ExecutorService executorService; // volatile to guarantee most current value is visible
private volatile ServerSocket serverSocket; // volatile to guarantee most current value is visible
private final int numThreads;
@ -96,7 +94,7 @@ public abstract class SocketListener {
@Override
public Thread newThread(final Runnable r) {
final Thread newThread = defaultThreadFactory.newThread(r);
newThread.setName("Process NCM Request-" + threadCounter.incrementAndGet());
newThread.setName("Process Cluster Protocol Request-" + threadCounter.incrementAndGet());
return newThread;
}
});
@ -152,6 +150,8 @@ public abstract class SocketListener {
});
t.setName("Cluster Socket Listener");
t.start();
logger.info("Now listening for connections from nodes on port " + port);
}
public boolean isRunning() {

View File

@ -435,7 +435,7 @@ public class StandardProcessorTestRunner implements TestRunner {
@Override
public void enqueue(final String data) {
enqueue(data.getBytes(StandardCharsets.UTF_8), Collections.emptyMap());
enqueue(data.getBytes(StandardCharsets.UTF_8), Collections.<String, String> emptyMap());
}
@Override

View File

@ -91,4 +91,22 @@ public class ComponentDTO {
public void setPosition(final PositionDTO position) {
this.position = position;
}
@Override
public int hashCode() {
return id == null ? 187 : 187 * id.hashCode();
}
@Override
public boolean equals(Object obj) {
if (obj == null) {
return false;
}
if (obj.getClass() != getClass()) {
return false;
}
return id.equals(((ComponentDTO) obj).getId());
}
}

View File

@ -112,4 +112,26 @@ public class ComponentEntity extends Entity {
public void setAccessPolicy(AccessPolicyDTO accessPolicy) {
this.accessPolicy = accessPolicy;
}
@Override
public int hashCode() {
return id.hashCode();
}
@Override
public boolean equals(Object obj) {
if (obj == null) {
return false;
}
if (obj == this) {
return true;
}
if (obj.getClass() != getClass()) {
return false;
}
return id.equals(((ComponentEntity) obj).getId());
}
}

View File

@ -24,8 +24,10 @@ import java.util.Set;
import org.apache.nifi.cluster.coordination.node.DisconnectionCode;
import org.apache.nifi.cluster.coordination.node.NodeConnectionState;
import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus;
import org.apache.nifi.cluster.event.NodeEvent;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.reporting.Severity;
import org.apache.nifi.services.FlowService;
/**
* <p>
@ -36,11 +38,18 @@ public interface ClusterCoordinator {
/**
* Sends a request to the node to connect to the cluster. This will immediately
* set the NodeConnectionStatus to DISCONNECTED.
* set the NodeConnectionStatus to CONNECTING.
*
* @param nodeId the identifier of the node
* @param userDn the DN of the user that requested that the node connect, or <code>null</code> if the action is not user-driven
*/
void requestNodeConnect(NodeIdentifier nodeId);
void requestNodeConnect(NodeIdentifier nodeId, String userDn);
/**
* Notifies the Cluster Coordinator that the NiFi instance is being shutdown so that
* the coordinator is able to perform cleanup of resources
*/
void shutdown();
/**
* Indicates that the node has sent a valid heartbeat and should now
@ -71,6 +80,14 @@ public interface ClusterCoordinator {
*/
void disconnectionRequestedByNode(NodeIdentifier nodeId, DisconnectionCode disconnectionCode, String explanation);
/**
* Removes the given disconnected node from the cluster
*
* @param nodeId the node to remove
* @param userDn the DN of the user requesting that the node be removed
*/
void removeNode(NodeIdentifier nodeId, String userDn);
/**
* Returns the current status of the node with the given identifier
*
@ -84,10 +101,10 @@ public interface ClusterCoordinator {
/**
* Returns the identifiers of all nodes that have the given connection state
*
* @param state the state
* @param states the states of interest
* @return the identifiers of all nodes that have the given connection state
*/
Set<NodeIdentifier> getNodeIdentifiers(NodeConnectionState state);
Set<NodeIdentifier> getNodeIdentifiers(NodeConnectionState... states);
/**
* Returns a Map of NodeConnectionStatus to all Node Identifiers that have that status.
@ -118,11 +135,12 @@ public interface ClusterCoordinator {
void reportEvent(NodeIdentifier nodeId, Severity severity, String event);
/**
* Updates the node that is considered the Primary Node
* Updates the roles held by the given node
*
* @param nodeId the id of the Primary Node
* @param nodeId the id of the node to update
* @param roles the new roles that the node possesses
*/
void setPrimaryNode(NodeIdentifier nodeId);
void updateNodeRoles(NodeIdentifier nodeId, Set<String> roles);
/**
* Returns the NodeIdentifier that exists that has the given UUID, or <code>null</code> if no NodeIdentifier
@ -133,4 +151,53 @@ public interface ClusterCoordinator {
* exists for the given UUID
*/
NodeIdentifier getNodeIdentifier(String uuid);
/**
* Returns all of the events that have occurred for the given node
*
* @param nodeId the identifier of the node
* @return all of the events that have occurred for the given node
*/
List<NodeEvent> getNodeEvents(NodeIdentifier nodeId);
/**
* @return the identifier of the node that is elected primary, or <code>null</code> if either there is no
* primary or the primary is not known by this node.
*/
NodeIdentifier getPrimaryNode();
/**
* Updates the Flow Service to use for obtaining the current flow
*
* @param flowService the flow service to use for obtaining the current flow
*/
void setFlowService(FlowService flowService);
/**
* Clears the current state of all nodes and replaces them with the values provided in the given map
*
* @param statusMap the new states of all nodes in the cluster
*/
void resetNodeStatuses(Map<NodeIdentifier, NodeConnectionStatus> statusMap);
/**
* Notifies the Cluster Coordinator of the Node Identifier that the coordinator is currently running on
*
* @param nodeId the ID of the current node
*/
void setLocalNodeIdentifier(NodeIdentifier nodeId);
/**
* Notifies the Cluster Coordinator whether or not the node is connected to the cluster
*
* @param connected <code>true</code> if the node is connected to a cluster, <code>false</code> otherwise.
*/
void setConnected(boolean connected);
/**
* Indicates whether or not the node is currently connected to the cluster
*
* @return <code>true</code> if connected, <code>false</code> otherwise
*/
boolean isConnected();
}

View File

@ -17,6 +17,8 @@
package org.apache.nifi.cluster.coordination.heartbeat;
import java.util.Set;
import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
@ -38,9 +40,9 @@ public interface NodeHeartbeat {
NodeConnectionStatus getConnectionStatus();
/**
* @return <code>true</code> if the node is the Primary Node in the cluster, <code>false</code> otherwise
* @return the set of Roles that the node currently possesses.
*/
boolean isPrimary();
Set<String> getRoles();
/**
* @return the number of FlowFiles that are queued up on the node

View File

@ -14,23 +14,12 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.flow;
/**
* Represents the various state of a flow managed by the cluster.
*
* The semantics of the values are:
* <ul>
* <li> CURRENT - the flow is current </li>
* <li> STALE - the flow is not current, but is eligible to be updated. </li>
* <li> UNKNOWN - the flow is not current and is not eligible to be updated.
* </li>
* </ul>
*
*/
public enum PersistedFlowState {
package org.apache.nifi.cluster.coordination.node;
CURRENT,
STALE,
UNKNOWN
public class ClusterRoles {
public static final String PRIMARY_NODE = "Primary Node";
public static final String CLUSTER_COORDINATOR = "Cluster Coordinator";
}

View File

@ -57,6 +57,16 @@ public enum DisconnectionCode {
*/
MISMATCHED_FLOWS("Node's Flow did not Match Cluster Flow"),
/**
* Cannot communicate with the node
*/
UNABLE_TO_COMMUNICATE("Unable to Communicate with Node"),
/**
* Node did not service a request that was replicated to it
*/
FAILED_TO_SERVICE_REQUEST("Failed to Service Request"),
/**
* Node is being shut down
*/

View File

@ -46,5 +46,11 @@ public enum NodeConnectionState {
* A node that is not connected to the cluster.
* A DISCONNECTED node can transition to CONNECTING.
*/
DISCONNECTED
DISCONNECTED,
/**
* A NodeConnectionState of REMOVED indicates that the node was removed from the cluster
* and is used in order to notify other nodes in the cluster.
*/
REMOVED;
}

View File

@ -17,8 +17,15 @@
package org.apache.nifi.cluster.coordination.node;
import java.util.Collections;
import java.util.HashSet;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;
import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.cluster.protocol.jaxb.message.NodeConnectionStatusAdapter;
/**
@ -26,33 +33,55 @@ import org.apache.nifi.cluster.protocol.jaxb.message.NodeConnectionStatusAdapter
*/
@XmlJavaTypeAdapter(NodeConnectionStatusAdapter.class)
public class NodeConnectionStatus {
private static final AtomicLong idGenerator = new AtomicLong(0L);
private final long updateId;
private final NodeIdentifier nodeId;
private final NodeConnectionState state;
private final DisconnectionCode disconnectCode;
private final String disconnectReason;
private final Long connectionRequestTime;
private final Set<String> roles;
public NodeConnectionStatus(final NodeConnectionState state) {
this(state, null, null, null);
public NodeConnectionStatus(final NodeIdentifier nodeId, final NodeConnectionState state) {
this(nodeId, state, null, null, null, null);
}
public NodeConnectionStatus(final NodeConnectionState state, final long connectionRequestTime) {
this(state, null, null, connectionRequestTime);
public NodeConnectionStatus(final NodeIdentifier nodeId, final NodeConnectionState state, final Set<String> roles) {
this(nodeId, state, null, null, null, roles);
}
public NodeConnectionStatus(final DisconnectionCode disconnectionCode) {
this(NodeConnectionState.DISCONNECTED, disconnectionCode, disconnectionCode.name(), null);
public NodeConnectionStatus(final NodeIdentifier nodeId, final NodeConnectionState state, final long connectionRequestTime) {
this(nodeId, state, null, null, connectionRequestTime, null);
}
public NodeConnectionStatus(final DisconnectionCode disconnectionCode, final String disconnectionExplanation) {
this(NodeConnectionState.DISCONNECTED, disconnectionCode, disconnectionExplanation, null);
public NodeConnectionStatus(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode) {
this(nodeId, NodeConnectionState.DISCONNECTED, disconnectionCode, disconnectionCode.name(), null, null);
}
public NodeConnectionStatus(final NodeConnectionState state, final DisconnectionCode disconnectionCode) {
this(state, disconnectionCode, disconnectionCode.name(), null);
public NodeConnectionStatus(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode, final String disconnectionExplanation) {
this(nodeId, NodeConnectionState.DISCONNECTED, disconnectionCode, disconnectionExplanation, null, null);
}
public NodeConnectionStatus(final NodeConnectionState state, final DisconnectionCode disconnectCode, final String disconnectReason, final Long connectionRequestTime) {
public NodeConnectionStatus(final NodeIdentifier nodeId, final NodeConnectionState state, final DisconnectionCode disconnectionCode, final Set<String> roles) {
this(nodeId, state, disconnectionCode, disconnectionCode.name(), null, roles);
}
public NodeConnectionStatus(final NodeConnectionStatus status, final Set<String> roles) {
this(status.getNodeIdentifier(), status.getState(), status.getDisconnectCode(), status.getDisconnectReason(), status.getConnectionRequestTime(), roles);
}
public NodeConnectionStatus(final NodeIdentifier nodeId, final NodeConnectionState state, final DisconnectionCode disconnectCode,
final String disconnectReason, final Long connectionRequestTime, final Set<String> roles) {
this(idGenerator.getAndIncrement(), nodeId, state, disconnectCode, disconnectReason, connectionRequestTime, roles);
}
public NodeConnectionStatus(final long updateId, final NodeIdentifier nodeId, final NodeConnectionState state, final DisconnectionCode disconnectCode,
final String disconnectReason, final Long connectionRequestTime, final Set<String> roles) {
this.updateId = updateId;
this.nodeId = nodeId;
this.state = state;
this.roles = roles == null ? Collections.emptySet() : Collections.unmodifiableSet(new HashSet<>(roles));
if (state == NodeConnectionState.DISCONNECTED && disconnectCode == null) {
this.disconnectCode = DisconnectionCode.UNKNOWN;
this.disconnectReason = this.disconnectCode.toString();
@ -61,7 +90,19 @@ public class NodeConnectionStatus {
this.disconnectReason = disconnectReason;
}
this.connectionRequestTime = connectionRequestTime;
this.connectionRequestTime = (connectionRequestTime == null && state == NodeConnectionState.CONNECTING) ? Long.valueOf(System.currentTimeMillis()) : connectionRequestTime;
}
public long getUpdateIdentifier() {
return updateId;
}
public Set<String> getRoles() {
return roles;
}
public NodeIdentifier getNodeIdentifier() {
return nodeId;
}
public NodeConnectionState getState() {
@ -88,7 +129,47 @@ public class NodeConnectionStatus {
if (state == NodeConnectionState.DISCONNECTED || state == NodeConnectionState.DISCONNECTING) {
sb.append(", Disconnect Code=").append(getDisconnectCode()).append(", Disconnect Reason=").append(getDisconnectReason());
}
sb.append(", updateId=").append(getUpdateIdentifier());
sb.append("]");
return sb.toString();
}
/**
* Updates the ID Generator so that it is at least equal to the given minimum value
*
* @param minimumValue the minimum value that the ID Generator should be set to
*/
static void updateIdGenerator(long minimumValue) {
idGenerator.updateAndGet(curValue -> Math.max(minimumValue, curValue));
}
@Override
public int hashCode() {
final int prime = 31;
int result = 1;
result = prime * result + ((nodeId == null) ? 0 : nodeId.hashCode());
result = prime * result + ((roles == null) ? 0 : roles.hashCode());
result = prime * result + ((state == null) ? 0 : state.hashCode());
return result;
}
@Override
public boolean equals(Object obj) {
if (this == obj) {
return true;
}
if (obj == null) {
return false;
}
if (!(obj instanceof NodeConnectionStatus)) {
return false;
}
NodeConnectionStatus other = (NodeConnectionStatus) obj;
return Objects.deepEquals(getNodeIdentifier(), other.getNodeIdentifier())
&& Objects.deepEquals(getRoles(), other.getRoles())
&& Objects.deepEquals(getState(), other.getState());
}
}

View File

@ -14,26 +14,17 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.flow;
/**
* A base exception for data access exceptions.
*
*/
public class DaoException extends RuntimeException {
package org.apache.nifi.cluster.event;
public DaoException() {
}
import org.apache.nifi.reporting.Severity;
public DaoException(String msg) {
super(msg);
}
public interface NodeEvent {
Severity getSeverity();
public DaoException(Throwable cause) {
super(cause);
}
String getMessage();
public DaoException(String msg, Throwable cause) {
super(msg, cause);
}
String getSource();
long getTimestamp();
}

View File

@ -14,18 +14,9 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.protocol.impl;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.Socket;
import java.security.cert.CertificateException;
import org.apache.nifi.cluster.protocol.NodeProtocolSender;
import org.apache.nifi.cluster.protocol.ProtocolContext;
import org.apache.nifi.cluster.protocol.ProtocolException;
import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller;
import org.apache.nifi.cluster.protocol.ProtocolMessageUnmarshaller;
import org.apache.nifi.cluster.protocol.UnknownServiceAddressException;
package org.apache.nifi.cluster.protocol;
import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage;
import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage;
import org.apache.nifi.cluster.protocol.message.HeartbeatMessage;
@ -33,26 +24,18 @@ import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType;
import org.apache.nifi.io.socket.SocketConfiguration;
import org.apache.nifi.io.socket.SocketUtils;
import org.apache.nifi.io.socket.multicast.DiscoverableService;
import org.apache.nifi.security.util.CertificateUtils;
public class NodeProtocolSenderImpl implements NodeProtocolSender {
import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.Socket;
import java.security.cert.CertificateException;
public abstract class AbstractNodeProtocolSender implements NodeProtocolSender {
private final SocketConfiguration socketConfiguration;
private final ClusterServiceLocator clusterManagerProtocolServiceLocator;
private final ProtocolContext<ProtocolMessage> protocolContext;
public NodeProtocolSenderImpl(final ClusterServiceLocator clusterManagerProtocolServiceLocator,
final SocketConfiguration socketConfiguration, final ProtocolContext<ProtocolMessage> protocolContext) {
if (clusterManagerProtocolServiceLocator == null) {
throw new IllegalArgumentException("Protocol Service Locator may not be null.");
} else if (socketConfiguration == null) {
throw new IllegalArgumentException("Socket configuration may not be null.");
} else if (protocolContext == null) {
throw new IllegalArgumentException("Protocol Context may not be null.");
}
this.clusterManagerProtocolServiceLocator = clusterManagerProtocolServiceLocator;
public AbstractNodeProtocolSender(final SocketConfiguration socketConfiguration, final ProtocolContext<ProtocolMessage> protocolContext) {
this.socketConfiguration = socketConfiguration;
this.protocolContext = protocolContext;
}
@ -63,7 +46,7 @@ public class NodeProtocolSenderImpl implements NodeProtocolSender {
try {
socket = createSocket();
String ncmDn = getNCMDN(socket);
String coordinatorDN = getCoordinatorDN(socket);
try {
// marshal message to output stream
@ -79,12 +62,13 @@ public class NodeProtocolSenderImpl implements NodeProtocolSender {
final ProtocolMessageUnmarshaller<ProtocolMessage> unmarshaller = protocolContext.createUnmarshaller();
response = unmarshaller.unmarshal(socket.getInputStream());
} catch (final IOException ioe) {
throw new ProtocolException("Failed unmarshalling '" + MessageType.CONNECTION_RESPONSE + "' protocol message due to: " + ioe, ioe);
throw new ProtocolException("Failed unmarshalling '" + MessageType.CONNECTION_RESPONSE + "' protocol message from "
+ socket.getRemoteSocketAddress() + " due to: " + ioe, ioe);
}
if (MessageType.CONNECTION_RESPONSE == response.getType()) {
final ConnectionResponseMessage connectionResponse = (ConnectionResponseMessage) response;
connectionResponse.setClusterManagerDN(ncmDn);
connectionResponse.setCoordinatorDN(coordinatorDN);
return connectionResponse;
} else {
throw new ProtocolException("Expected message type '" + MessageType.CONNECTION_RESPONSE + "' but found '" + response.getType() + "'");
@ -109,7 +93,7 @@ public class NodeProtocolSenderImpl implements NodeProtocolSender {
sendProtocolMessage(msg, hostname, port);
}
private String getNCMDN(Socket socket) {
private String getCoordinatorDN(Socket socket) {
try {
return CertificateUtils.extractClientDNFromSSLSocket(socket);
} catch (CertificateException e) {
@ -118,17 +102,17 @@ public class NodeProtocolSenderImpl implements NodeProtocolSender {
}
private Socket createSocket() {
// determine the cluster manager's address
final DiscoverableService service = clusterManagerProtocolServiceLocator.getService();
if (service == null) {
throw new UnknownServiceAddressException("Cluster Manager's service is not known. Verify a cluster manager is running.");
}
InetSocketAddress socketAddress = null;
try {
// create a socket
return SocketUtils.createSocket(service.getServiceAddress(), socketConfiguration);
socketAddress = getServiceAddress();
return SocketUtils.createSocket(socketAddress, socketConfiguration);
} catch (final IOException ioe) {
throw new ProtocolException("Failed to create socket due to: " + ioe, ioe);
if (socketAddress == null) {
throw new ProtocolException("Failed to create socket due to: " + ioe, ioe);
} else {
throw new ProtocolException("Failed to create socket to " + socketAddress + " due to: " + ioe, ioe);
}
}
}
@ -156,4 +140,6 @@ public class NodeProtocolSenderImpl implements NodeProtocolSender {
SocketUtils.closeQuietly(socket);
}
}
protected abstract InetSocketAddress getServiceAddress() throws IOException;
}

View File

@ -19,8 +19,6 @@ package org.apache.nifi.cluster.protocol;
import java.util.Set;
import org.apache.nifi.cluster.protocol.message.DisconnectMessage;
import org.apache.nifi.cluster.protocol.message.FlowRequestMessage;
import org.apache.nifi.cluster.protocol.message.FlowResponseMessage;
import org.apache.nifi.cluster.protocol.message.NodeStatusChangeMessage;
import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage;
import org.apache.nifi.cluster.protocol.message.ReconnectionResponseMessage;
@ -30,16 +28,7 @@ import org.apache.nifi.reporting.BulletinRepository;
* An interface for sending protocol messages from the cluster manager to nodes.
*
*/
public interface ClusterManagerProtocolSender {
/**
* Sends a "flow request" message to a node.
*
* @param msg a message
* @return the response
* @throws ProtocolException if communication failed
*/
FlowResponseMessage requestFlow(FlowRequestMessage msg) throws ProtocolException;
public interface ClusterCoordinationProtocolSender {
/**
* Sends a "reconnection request" message to a node.

View File

@ -0,0 +1,104 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.protocol;
import org.apache.nifi.web.Revision;
public class ComponentRevision {
private Long version;
private String clientId;
private String componentId;
public Long getVersion() {
return version;
}
public void setVersion(Long version) {
this.version = version;
}
public String getClientId() {
return clientId;
}
public void setClientId(String clientId) {
this.clientId = clientId;
}
public String getComponentId() {
return componentId;
}
public void setComponentId(String componentId) {
this.componentId = componentId;
}
public Revision toRevision() {
return new Revision(getVersion(), getClientId(), getComponentId());
}
public static ComponentRevision fromRevision(final Revision revision) {
final ComponentRevision componentRevision = new ComponentRevision();
componentRevision.setVersion(revision.getVersion());
componentRevision.setClientId(revision.getClientId());
componentRevision.setComponentId(revision.getComponentId());
return componentRevision;
}
@Override
public boolean equals(final Object obj) {
if (obj == null) {
return false;
}
if (obj == this) {
return true;
}
if ((obj instanceof ComponentRevision) == false) {
return false;
}
ComponentRevision thatRevision = (ComponentRevision) obj;
// ensure that component ID's are the same (including null)
if (thatRevision.getComponentId() == null && getComponentId() != null) {
return false;
}
if (thatRevision.getComponentId() != null && getComponentId() == null) {
return false;
}
if (thatRevision.getComponentId() != null && !thatRevision.getComponentId().equals(getComponentId())) {
return false;
}
if (this.version != null && this.version.equals(thatRevision.version)) {
return true;
} else {
return clientId != null && !clientId.trim().isEmpty() && clientId.equals(thatRevision.getClientId());
}
}
@Override
public int hashCode() {
int hash = 5;
hash = 59 * hash + (this.componentId != null ? this.componentId.hashCode() : 0);
hash = 59 * hash + (this.version != null ? this.version.hashCode() : 0);
hash = 59 * hash + (this.clientId != null ? this.clientId.hashCode() : 0);
return hash;
}
}

View File

@ -16,10 +16,14 @@
*/
package org.apache.nifi.cluster.protocol;
import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus;
import org.apache.nifi.cluster.protocol.jaxb.message.ConnectionResponseAdapter;
import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
/**
* The cluster manager's response to a node's connection request. If the manager
* has a current copy of the data flow, then it is returned with a node
@ -34,15 +38,19 @@ public class ConnectionResponse {
private final String rejectionReason;
private final int tryLaterSeconds;
private final NodeIdentifier nodeIdentifier;
private final StandardDataFlow dataFlow;
private final DataFlow dataFlow;
private final Integer managerRemoteInputPort;
private final Boolean managerRemoteCommsSecure;
private final String instanceId;
private final List<NodeConnectionStatus> nodeStatuses;
private final List<ComponentRevision> componentRevisions;
private volatile String clusterManagerDN;
private volatile String coordinatorDN;
public ConnectionResponse(final NodeIdentifier nodeIdentifier, final DataFlow dataFlow,
final Integer managerRemoteInputPort, final Boolean managerRemoteCommsSecure, final String instanceId,
final List<NodeConnectionStatus> nodeStatuses, final List<ComponentRevision> componentRevisions) {
public ConnectionResponse(final NodeIdentifier nodeIdentifier, final StandardDataFlow dataFlow,
final Integer managerRemoteInputPort, final Boolean managerRemoteCommsSecure, final String instanceId) {
if (nodeIdentifier == null) {
throw new IllegalArgumentException("Node identifier may not be empty or null.");
} else if (dataFlow == null) {
@ -55,6 +63,8 @@ public class ConnectionResponse {
this.managerRemoteInputPort = managerRemoteInputPort;
this.managerRemoteCommsSecure = managerRemoteCommsSecure;
this.instanceId = instanceId;
this.nodeStatuses = Collections.unmodifiableList(new ArrayList<>(nodeStatuses));
this.componentRevisions = componentRevisions == null ? Collections.emptyList() : Collections.unmodifiableList(new ArrayList<>(componentRevisions));
}
public ConnectionResponse(final int tryLaterSeconds) {
@ -68,6 +78,8 @@ public class ConnectionResponse {
this.managerRemoteInputPort = null;
this.managerRemoteCommsSecure = null;
this.instanceId = null;
this.nodeStatuses = null;
this.componentRevisions = null;
}
private ConnectionResponse(final String rejectionReason) {
@ -78,6 +90,8 @@ public class ConnectionResponse {
this.managerRemoteInputPort = null;
this.managerRemoteCommsSecure = null;
this.instanceId = null;
this.nodeStatuses = null;
this.componentRevisions = null;
}
public static ConnectionResponse createBlockedByFirewallResponse() {
@ -104,7 +118,7 @@ public class ConnectionResponse {
return tryLaterSeconds;
}
public StandardDataFlow getDataFlow() {
public DataFlow getDataFlow() {
return dataFlow;
}
@ -124,15 +138,23 @@ public class ConnectionResponse {
return instanceId;
}
public void setClusterManagerDN(final String dn) {
this.clusterManagerDN = dn;
public void setCoordinatorDN(final String dn) {
this.coordinatorDN = dn;
}
public List<NodeConnectionStatus> getNodeConnectionStatuses() {
return nodeStatuses;
}
public List<ComponentRevision> getComponentRevisions() {
return componentRevisions;
}
/**
* @return the DN of the NCM, if it is available or <code>null</code>
* @return the DN of the Coordinator, if it is available or <code>null</code>
* otherwise
*/
public String getClusterManagerDN() {
return clusterManagerDN;
public String getCoordinatorDN() {
return coordinatorDN;
}
}

View File

@ -16,7 +16,11 @@
*/
package org.apache.nifi.cluster.protocol;
import java.util.Collections;
import java.util.Date;
import java.util.HashSet;
import java.util.Set;
import javax.xml.bind.annotation.XmlTransient;
import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
@ -31,17 +35,17 @@ import org.apache.nifi.cluster.protocol.jaxb.message.HeartbeatAdapter;
public class Heartbeat {
private final NodeIdentifier nodeIdentifier;
private final boolean primary;
private final Set<String> roles;
private final NodeConnectionStatus connectionStatus;
private final long createdTimestamp;
private final byte[] payload;
public Heartbeat(final NodeIdentifier nodeIdentifier, final boolean primary, final NodeConnectionStatus connectionStatus, final byte[] payload) {
public Heartbeat(final NodeIdentifier nodeIdentifier, final Set<String> roles, final NodeConnectionStatus connectionStatus, final byte[] payload) {
if (nodeIdentifier == null) {
throw new IllegalArgumentException("Node Identifier may not be null.");
}
this.nodeIdentifier = nodeIdentifier;
this.primary = primary;
this.roles = roles == null ? Collections.emptySet() : Collections.unmodifiableSet(new HashSet<>(roles));
this.connectionStatus = connectionStatus;
this.payload = payload;
this.createdTimestamp = new Date().getTime();
@ -55,8 +59,8 @@ public class Heartbeat {
return payload;
}
public boolean isPrimary() {
return primary;
public Set<String> getRoles() {
return roles;
}
public NodeConnectionStatus getConnectionStatus() {

View File

@ -20,14 +20,12 @@ import java.io.IOException;
import java.util.Collection;
import java.util.Set;
import org.apache.nifi.cluster.protocol.ClusterManagerProtocolSender;
import org.apache.nifi.cluster.protocol.ClusterCoordinationProtocolSender;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.cluster.protocol.ProtocolException;
import org.apache.nifi.cluster.protocol.ProtocolHandler;
import org.apache.nifi.cluster.protocol.ProtocolListener;
import org.apache.nifi.cluster.protocol.message.DisconnectMessage;
import org.apache.nifi.cluster.protocol.message.FlowRequestMessage;
import org.apache.nifi.cluster.protocol.message.FlowResponseMessage;
import org.apache.nifi.cluster.protocol.message.NodeStatusChangeMessage;
import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage;
import org.apache.nifi.cluster.protocol.message.ReconnectionResponseMessage;
@ -38,13 +36,13 @@ import org.apache.nifi.reporting.BulletinRepository;
* cluster manager.
*
*/
public class ClusterManagerProtocolSenderListener implements ClusterManagerProtocolSender, ProtocolListener {
public class ClusterCoordinationProtocolSenderListener implements ClusterCoordinationProtocolSender, ProtocolListener {
private final ClusterManagerProtocolSender sender;
private final ClusterCoordinationProtocolSender sender;
private final ProtocolListener listener;
public ClusterManagerProtocolSenderListener(final ClusterManagerProtocolSender sender, final ProtocolListener listener) {
public ClusterCoordinationProtocolSenderListener(final ClusterCoordinationProtocolSender sender, final ProtocolListener listener) {
if (sender == null) {
throw new IllegalArgumentException("ClusterManagerProtocolSender may not be null.");
} else if (listener == null) {
@ -96,11 +94,6 @@ public class ClusterManagerProtocolSenderListener implements ClusterManagerProto
sender.setBulletinRepository(bulletinRepository);
}
@Override
public FlowResponseMessage requestFlow(final FlowRequestMessage msg) throws ProtocolException {
return sender.requestFlow(msg);
}
@Override
public ReconnectionResponseMessage requestReconnection(final ReconnectionRequestMessage msg) throws ProtocolException {
return sender.requestReconnection(msg);

View File

@ -1,180 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.protocol.impl;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.Collection;
import java.util.Collections;
import org.apache.nifi.io.socket.multicast.DiscoverableService;
import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl;
import org.apache.nifi.io.socket.multicast.MulticastConfiguration;
import org.apache.nifi.io.socket.multicast.MulticastServiceDiscovery;
import org.apache.nifi.reporting.BulletinRepository;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.cluster.protocol.ProtocolContext;
import org.apache.nifi.cluster.protocol.ProtocolException;
import org.apache.nifi.cluster.protocol.ProtocolHandler;
import org.apache.nifi.cluster.protocol.ProtocolListener;
import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType;
import org.apache.nifi.cluster.protocol.message.ServiceBroadcastMessage;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* An implementation for discovering services by way of "service broadcast" type
* protocol messages over multicast.
*
* The client caller is responsible for starting and stopping the service
* discovery. The instance must be stopped before termination of the JVM to
* ensure proper resource clean-up.
*
*/
public class ClusterServiceDiscovery implements MulticastServiceDiscovery, ProtocolListener {
private static final Logger logger = LoggerFactory.getLogger(ClusterServiceDiscovery.class);
private final String serviceName;
private final MulticastConfiguration multicastConfiguration;
private final MulticastProtocolListener listener;
private volatile BulletinRepository bulletinRepository;
/*
* guarded by this
*/
private DiscoverableService service;
public ClusterServiceDiscovery(final String serviceName, final InetSocketAddress multicastAddress,
final MulticastConfiguration multicastConfiguration, final ProtocolContext<ProtocolMessage> protocolContext) {
if (StringUtils.isBlank(serviceName)) {
throw new IllegalArgumentException("Service name may not be null or empty.");
} else if (multicastAddress == null) {
throw new IllegalArgumentException("Multicast address may not be null.");
} else if (multicastAddress.getAddress().isMulticastAddress() == false) {
throw new IllegalArgumentException("Multicast group must be a Class D address.");
} else if (protocolContext == null) {
throw new IllegalArgumentException("Protocol Context may not be null.");
} else if (multicastConfiguration == null) {
throw new IllegalArgumentException("Multicast configuration may not be null.");
}
this.serviceName = serviceName;
this.multicastConfiguration = multicastConfiguration;
this.listener = new MulticastProtocolListener(1, multicastAddress, multicastConfiguration, protocolContext);
listener.addHandler(new ClusterManagerServiceBroadcastHandler());
}
@Override
public void setBulletinRepository(final BulletinRepository bulletinRepository) {
this.bulletinRepository = bulletinRepository;
}
@Override
public synchronized DiscoverableService getService() {
return service;
}
@Override
public InetSocketAddress getMulticastAddress() {
return listener.getMulticastAddress();
}
@Override
public Collection<ProtocolHandler> getHandlers() {
return Collections.unmodifiableCollection(listener.getHandlers());
}
@Override
public void addHandler(ProtocolHandler handler) {
listener.addHandler(handler);
}
@Override
public boolean removeHandler(ProtocolHandler handler) {
return listener.removeHandler(handler);
}
@Override
public boolean isRunning() {
return listener.isRunning();
}
@Override
public void start() throws IOException {
if (isRunning()) {
throw new IllegalStateException("Instance is already running.");
}
listener.start();
}
@Override
public void stop() throws IOException {
if (isRunning() == false) {
throw new IllegalStateException("Instance is already stopped.");
}
listener.stop();
}
public String getServiceName() {
return serviceName;
}
public MulticastConfiguration getMulticastConfiguration() {
return multicastConfiguration;
}
private class ClusterManagerServiceBroadcastHandler implements ProtocolHandler {
@Override
public boolean canHandle(final ProtocolMessage msg) {
return MessageType.SERVICE_BROADCAST == msg.getType();
}
@Override
public ProtocolMessage handle(final ProtocolMessage msg) throws ProtocolException {
synchronized (ClusterServiceDiscovery.this) {
if (canHandle(msg) == false) {
throw new ProtocolException("Handler cannot handle message type: " + msg.getType());
} else {
final ServiceBroadcastMessage broadcastMsg = (ServiceBroadcastMessage) msg;
if (serviceName.equals(broadcastMsg.getServiceName())) {
final DiscoverableService oldService = service;
if (oldService == null
|| broadcastMsg.getAddress().equalsIgnoreCase(oldService.getServiceAddress().getHostName()) == false
|| broadcastMsg.getPort() != oldService.getServiceAddress().getPort()) {
service = new DiscoverableServiceImpl(serviceName, InetSocketAddress.createUnresolved(broadcastMsg.getAddress(), broadcastMsg.getPort()));
final InetSocketAddress oldServiceAddress = (oldService == null) ? null : oldService.getServiceAddress();
logger.info(String.format("Updating cluster service address for '%s' from '%s' to '%s'",
serviceName, prettyPrint(oldServiceAddress), prettyPrint(service.getServiceAddress())));
}
}
return null;
}
}
}
}
private String prettyPrint(final InetSocketAddress address) {
if (address == null) {
return "0.0.0.0:0";
} else {
return address.getHostName() + ":" + address.getPort();
}
}
}

View File

@ -1,228 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.protocol.impl;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.nifi.io.socket.multicast.DiscoverableService;
import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl;
import org.apache.nifi.io.socket.multicast.ServiceDiscovery;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Implements the ServiceLocator interface for locating the socket address of a
* cluster service. Depending on configuration, the address may be located using
* service discovery. If using service discovery, then the service methods must
* be used for starting and stopping discovery.
*
* Service discovery may be used in conjunction with a fixed port. In this case,
* the service discovery will yield the service IP/host while the fixed port
* will be used for the port.
*
* Alternatively, the instance may be configured with exact service location, in
* which case, no service discovery occurs and the caller will always receive
* the configured service.
*
*/
public class ClusterServiceLocator implements ServiceDiscovery {
private static final Logger logger = LoggerFactory.getLogger(ClusterServiceLocator.class);
private final String serviceName;
private final ClusterServiceDiscovery serviceDiscovery;
private final DiscoverableService fixedService;
private final int fixedServicePort;
private final AttemptsConfig attemptsConfig = new AttemptsConfig();
private final AtomicBoolean running = new AtomicBoolean(false);
public ClusterServiceLocator(final ClusterServiceDiscovery serviceDiscovery) {
if (serviceDiscovery == null) {
throw new IllegalArgumentException("ClusterServiceDiscovery may not be null.");
}
this.serviceDiscovery = serviceDiscovery;
this.fixedService = null;
this.fixedServicePort = 0;
this.serviceName = serviceDiscovery.getServiceName();
}
public ClusterServiceLocator(final ClusterServiceDiscovery serviceDiscovery, final int fixedServicePort) {
if (serviceDiscovery == null) {
throw new IllegalArgumentException("ClusterServiceDiscovery may not be null.");
}
this.serviceDiscovery = serviceDiscovery;
this.fixedService = null;
this.fixedServicePort = fixedServicePort;
this.serviceName = serviceDiscovery.getServiceName();
}
public ClusterServiceLocator(final DiscoverableService fixedService) {
if (fixedService == null) {
throw new IllegalArgumentException("Service may not be null.");
}
this.serviceDiscovery = null;
this.fixedService = fixedService;
this.fixedServicePort = 0;
this.serviceName = fixedService.getServiceName();
}
@Override
public DiscoverableService getService() {
final int numAttemptsValue;
final int secondsBetweenAttempts;
synchronized (this) {
numAttemptsValue = attemptsConfig.numAttempts;
secondsBetweenAttempts = attemptsConfig.getTimeBetweenAttempts();
}
// try for a configured amount of attempts to retrieve the service address
for (int i = 0; i < numAttemptsValue; i++) {
if (fixedService != null) {
return fixedService;
} else if (serviceDiscovery != null) {
final DiscoverableService discoveredService = serviceDiscovery.getService();
// if we received an address
if (discoveredService != null) {
// if we were configured with a fixed port, then use the discovered host and fixed port; otherwise use the discovered address
if (fixedServicePort > 0) {
// create service using discovered service name and address with fixed service port
final InetSocketAddress addr = InetSocketAddress.createUnresolved(discoveredService.getServiceAddress().getHostName(), fixedServicePort);
final DiscoverableService result = new DiscoverableServiceImpl(discoveredService.getServiceName(), addr);
return result;
} else {
return discoveredService;
}
}
}
// could not obtain service address, so sleep a bit
try {
logger.debug(String.format("Locating Cluster Service '%s' Attempt: %d of %d failed. Trying again in %d seconds.",
serviceName, (i + 1), numAttemptsValue, secondsBetweenAttempts));
Thread.sleep(secondsBetweenAttempts * 1000);
} catch (final InterruptedException ie) {
break;
}
}
return null;
}
public boolean isRunning() {
if (serviceDiscovery != null) {
return serviceDiscovery.isRunning();
} else {
return running.get();
}
}
public void start() throws IOException {
if (isRunning()) {
throw new IllegalStateException("Instance is already started.");
}
if (serviceDiscovery != null) {
serviceDiscovery.start();
}
running.set(true);
}
public void stop() throws IOException {
if (isRunning() == false) {
throw new IllegalStateException("Instance is already stopped.");
}
if (serviceDiscovery != null) {
serviceDiscovery.stop();
}
running.set(false);
}
public synchronized void setAttemptsConfig(final AttemptsConfig config) {
if (config == null) {
throw new IllegalArgumentException("Attempts configuration may not be null.");
}
this.attemptsConfig.numAttempts = config.numAttempts;
this.attemptsConfig.timeBetweenAttempts = config.timeBetweenAttempts;
this.attemptsConfig.timeBetweenAttempsUnit = config.timeBetweenAttempsUnit;
}
public synchronized AttemptsConfig getAttemptsConfig() {
final AttemptsConfig config = new AttemptsConfig();
config.numAttempts = this.attemptsConfig.numAttempts;
config.timeBetweenAttempts = this.attemptsConfig.timeBetweenAttempts;
config.timeBetweenAttempsUnit = this.attemptsConfig.timeBetweenAttempsUnit;
return config;
}
public static class AttemptsConfig {
private int numAttempts = 1;
private int timeBetweenAttempts = 1;
private TimeUnit timeBetweenAttempsUnit = TimeUnit.SECONDS;
public int getNumAttempts() {
return numAttempts;
}
public void setNumAttempts(int numAttempts) {
if (numAttempts <= 0) {
throw new IllegalArgumentException("Number of attempts must be positive: " + numAttempts);
}
this.numAttempts = numAttempts;
}
public TimeUnit getTimeBetweenAttemptsUnit() {
return timeBetweenAttempsUnit;
}
public void setTimeBetweenAttempsUnit(TimeUnit timeBetweenAttempsUnit) {
if (timeBetweenAttempts <= 0) {
throw new IllegalArgumentException("Time between attempts must be positive: " + numAttempts);
}
this.timeBetweenAttempsUnit = timeBetweenAttempsUnit;
}
public int getTimeBetweenAttempts() {
return timeBetweenAttempts;
}
public void setTimeBetweenAttempts(int timeBetweenAttempts) {
if (timeBetweenAttempts <= 0) {
throw new IllegalArgumentException("Time between attempts must be positive: " + numAttempts);
}
this.timeBetweenAttempts = timeBetweenAttempts;
}
}
}

View File

@ -1,185 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.protocol.impl;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.net.DatagramPacket;
import java.net.InetSocketAddress;
import java.net.MulticastSocket;
import java.util.Collections;
import java.util.Set;
import java.util.Timer;
import java.util.TimerTask;
import java.util.concurrent.CopyOnWriteArraySet;
import java.util.concurrent.TimeUnit;
import org.apache.nifi.io.socket.multicast.DiscoverableService;
import org.apache.nifi.io.socket.multicast.MulticastConfiguration;
import org.apache.nifi.io.socket.multicast.MulticastServicesBroadcaster;
import org.apache.nifi.io.socket.multicast.MulticastUtils;
import org.apache.nifi.logging.NiFiLog;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.cluster.protocol.ProtocolContext;
import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller;
import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
import org.apache.nifi.cluster.protocol.message.ServiceBroadcastMessage;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Broadcasts services used by the clustering software using multicast
* communication. A configurable delay occurs after broadcasting the collection
* of services.
*
* The client caller is responsible for starting and stopping the broadcasting.
* The instance must be stopped before termination of the JVM to ensure proper
* resource clean-up.
*
*/
public class ClusterServicesBroadcaster implements MulticastServicesBroadcaster {
private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(ClusterServicesBroadcaster.class));
private final Set<DiscoverableService> services = new CopyOnWriteArraySet<>();
private final InetSocketAddress multicastAddress;
private final MulticastConfiguration multicastConfiguration;
private final ProtocolContext<ProtocolMessage> protocolContext;
private final int broadcastDelayMs;
private Timer broadcaster;
private MulticastSocket multicastSocket;
public ClusterServicesBroadcaster(final InetSocketAddress multicastAddress,
final MulticastConfiguration multicastConfiguration,
final ProtocolContext<ProtocolMessage> protocolContext, final String broadcastDelay) {
if (multicastAddress == null) {
throw new IllegalArgumentException("Multicast address may not be null.");
} else if (multicastAddress.getAddress().isMulticastAddress() == false) {
throw new IllegalArgumentException("Multicast group address is not a Class D IP address.");
} else if (protocolContext == null) {
throw new IllegalArgumentException("Protocol Context may not be null.");
} else if (multicastConfiguration == null) {
throw new IllegalArgumentException("Multicast configuration may not be null.");
}
this.services.addAll(services);
this.multicastAddress = multicastAddress;
this.multicastConfiguration = multicastConfiguration;
this.protocolContext = protocolContext;
this.broadcastDelayMs = (int) FormatUtils.getTimeDuration(broadcastDelay, TimeUnit.MILLISECONDS);
}
public void start() throws IOException {
if (isRunning()) {
throw new IllegalStateException("Instance is already started.");
}
// setup socket
multicastSocket = MulticastUtils.createMulticastSocket(multicastConfiguration);
// setup broadcaster
broadcaster = new Timer("Cluster Services Broadcaster", /* is daemon */ true);
broadcaster.schedule(new TimerTask() {
@Override
public void run() {
for (final DiscoverableService service : services) {
try {
final InetSocketAddress serviceAddress = service.getServiceAddress();
logger.debug(String.format("Broadcasting Cluster Service '%s' at address %s:%d",
service.getServiceName(), serviceAddress.getHostName(), serviceAddress.getPort()));
// create message
final ServiceBroadcastMessage msg = new ServiceBroadcastMessage();
msg.setServiceName(service.getServiceName());
msg.setAddress(serviceAddress.getHostName());
msg.setPort(serviceAddress.getPort());
// marshal message to output stream
final ProtocolMessageMarshaller<ProtocolMessage> marshaller = protocolContext.createMarshaller();
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
marshaller.marshal(msg, baos);
final byte[] packetBytes = baos.toByteArray();
// send message
final DatagramPacket packet = new DatagramPacket(packetBytes, packetBytes.length, multicastAddress);
multicastSocket.send(packet);
} catch (final Exception ex) {
logger.warn(String.format("Cluster Services Broadcaster failed broadcasting service '%s' due to: %s", service.getServiceName(), ex), ex);
}
}
}
}, 0, broadcastDelayMs);
}
public boolean isRunning() {
return (broadcaster != null);
}
public void stop() {
if (isRunning() == false) {
throw new IllegalStateException("Instance is already stopped.");
}
broadcaster.cancel();
broadcaster = null;
// close socket
MulticastUtils.closeQuietly(multicastSocket);
}
@Override
public int getBroadcastDelayMs() {
return broadcastDelayMs;
}
@Override
public Set<DiscoverableService> getServices() {
return Collections.unmodifiableSet(services);
}
@Override
public InetSocketAddress getMulticastAddress() {
return multicastAddress;
}
@Override
public boolean addService(final DiscoverableService service) {
return services.add(service);
}
@Override
public boolean removeService(final String serviceName) {
for (final DiscoverableService service : services) {
if (service.getServiceName().equals(serviceName)) {
return services.remove(service);
}
}
return false;
}
}

View File

@ -1,203 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.protocol.impl;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.net.DatagramPacket;
import java.net.InetSocketAddress;
import java.net.MulticastSocket;
import java.util.Collection;
import java.util.Collections;
import java.util.UUID;
import java.util.concurrent.CopyOnWriteArrayList;
import org.apache.nifi.io.socket.multicast.MulticastConfiguration;
import org.apache.nifi.io.socket.multicast.MulticastListener;
import org.apache.nifi.logging.NiFiLog;
import org.apache.nifi.reporting.Bulletin;
import org.apache.nifi.reporting.BulletinRepository;
import org.apache.nifi.cluster.protocol.ProtocolContext;
import org.apache.nifi.cluster.protocol.ProtocolException;
import org.apache.nifi.cluster.protocol.ProtocolHandler;
import org.apache.nifi.cluster.protocol.ProtocolListener;
import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller;
import org.apache.nifi.cluster.protocol.ProtocolMessageUnmarshaller;
import org.apache.nifi.cluster.protocol.message.MulticastProtocolMessage;
import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
import org.apache.nifi.events.BulletinFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Implements a listener for protocol messages sent over multicast. If a message
* is of type MulticastProtocolMessage, then the underlying protocol message is
* passed to the handler. If the receiving handler produces a message response,
* then the message is wrapped with a MulticastProtocolMessage before being sent
* to the originator.
*
* The client caller is responsible for starting and stopping the listener. The
* instance must be stopped before termination of the JVM to ensure proper
* resource clean-up.
*
*/
public class MulticastProtocolListener extends MulticastListener implements ProtocolListener {
private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(MulticastProtocolListener.class));
// immutable members
private final Collection<ProtocolHandler> handlers = new CopyOnWriteArrayList<>();
private final String listenerId = UUID.randomUUID().toString();
private final ProtocolContext<ProtocolMessage> protocolContext;
private volatile BulletinRepository bulletinRepository;
public MulticastProtocolListener(
final int numThreads,
final InetSocketAddress multicastAddress,
final MulticastConfiguration configuration,
final ProtocolContext<ProtocolMessage> protocolContext) {
super(numThreads, multicastAddress, configuration);
if (protocolContext == null) {
throw new IllegalArgumentException("Protocol Context may not be null.");
}
this.protocolContext = protocolContext;
}
@Override
public void setBulletinRepository(final BulletinRepository bulletinRepository) {
this.bulletinRepository = bulletinRepository;
}
@Override
public void start() throws IOException {
if (super.isRunning()) {
throw new IllegalStateException("Instance is already started.");
}
super.start();
}
@Override
public void stop() throws IOException {
if (super.isRunning() == false) {
throw new IllegalStateException("Instance is already stopped.");
}
// shutdown listener
super.stop();
}
@Override
public Collection<ProtocolHandler> getHandlers() {
return Collections.unmodifiableCollection(handlers);
}
@Override
public void addHandler(final ProtocolHandler handler) {
if (handler == null) {
throw new NullPointerException("Protocol handler may not be null.");
}
handlers.add(handler);
}
@Override
public boolean removeHandler(final ProtocolHandler handler) {
return handlers.remove(handler);
}
@Override
public void dispatchRequest(final MulticastSocket multicastSocket, final DatagramPacket packet) {
try {
// unmarshall message
final ProtocolMessageUnmarshaller<ProtocolMessage> unmarshaller = protocolContext.createUnmarshaller();
final ProtocolMessage request = unmarshaller.unmarshal(new ByteArrayInputStream(packet.getData(), 0, packet.getLength()));
// unwrap multicast message, if necessary
final ProtocolMessage unwrappedRequest;
if (request instanceof MulticastProtocolMessage) {
final MulticastProtocolMessage multicastRequest = (MulticastProtocolMessage) request;
// don't process a message we sent
if (listenerId.equals(multicastRequest.getId())) {
return;
} else {
unwrappedRequest = multicastRequest.getProtocolMessage();
}
} else {
unwrappedRequest = request;
}
// dispatch message to handler
ProtocolHandler desiredHandler = null;
for (final ProtocolHandler handler : getHandlers()) {
if (handler.canHandle(unwrappedRequest)) {
desiredHandler = handler;
break;
}
}
// if no handler found, throw exception; otherwise handle request
if (desiredHandler == null) {
throw new ProtocolException("No handler assigned to handle message type: " + request.getType());
} else {
final ProtocolMessage response = desiredHandler.handle(request);
if (response != null) {
try {
// wrap with listener id
final MulticastProtocolMessage multicastResponse = new MulticastProtocolMessage(listenerId, response);
// marshal message
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
final ProtocolMessageMarshaller<ProtocolMessage> marshaller = protocolContext.createMarshaller();
marshaller.marshal(multicastResponse, baos);
final byte[] responseBytes = baos.toByteArray();
final int maxPacketSizeBytes = getMaxPacketSizeBytes();
if (responseBytes.length > maxPacketSizeBytes) {
logger.warn("Cluster protocol handler '" + desiredHandler.getClass()
+ "' produced a multicast response with length greater than configured max packet size '" + maxPacketSizeBytes + "'");
}
// create and send packet
final DatagramPacket responseDatagram = new DatagramPacket(responseBytes, responseBytes.length, getMulticastAddress().getAddress(), getMulticastAddress().getPort());
multicastSocket.send(responseDatagram);
} catch (final IOException ioe) {
throw new ProtocolException("Failed marshalling protocol message in response to message type: " + request.getType() + " due to: " + ioe, ioe);
}
}
}
} catch (final Throwable t) {
logger.warn("Failed processing protocol message due to " + t, t);
if (bulletinRepository != null) {
final Bulletin bulletin = BulletinFactory.createBulletin("Clustering", "WARNING", "Failed to process Protocol Message due to " + t.toString());
bulletinRepository.addBulletin(bulletin);
}
}
}
}

View File

@ -30,7 +30,6 @@ import org.apache.nifi.cluster.protocol.message.HeartbeatMessage;
import org.apache.nifi.reporting.BulletinRepository;
public class NodeProtocolSenderListener implements NodeProtocolSender, ProtocolListener {
private final NodeProtocolSender sender;
private final ProtocolListener listener;

View File

@ -25,6 +25,7 @@ import java.util.Collections;
import java.util.UUID;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.TimeUnit;
import org.apache.nifi.cluster.protocol.ProtocolContext;
import org.apache.nifi.cluster.protocol.ProtocolException;
import org.apache.nifi.cluster.protocol.ProtocolHandler;
@ -35,7 +36,6 @@ import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
import org.apache.nifi.events.BulletinFactory;
import org.apache.nifi.io.socket.ServerSocketConfiguration;
import org.apache.nifi.io.socket.SocketListener;
import org.apache.nifi.logging.NiFiLog;
import org.apache.nifi.reporting.Bulletin;
import org.apache.nifi.reporting.BulletinRepository;
import org.apache.nifi.security.util.CertificateUtils;
@ -49,7 +49,7 @@ import org.slf4j.LoggerFactory;
*/
public class SocketProtocolListener extends SocketListener implements ProtocolListener {
private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(SocketProtocolListener.class));
private static final Logger logger = LoggerFactory.getLogger(SocketProtocolListener.class);
private final ProtocolContext<ProtocolMessage> protocolContext;
private final Collection<ProtocolHandler> handlers = new CopyOnWriteArrayList<>();
private volatile BulletinRepository bulletinRepository;
@ -122,7 +122,7 @@ public class SocketProtocolListener extends SocketListener implements ProtocolLi
final StopWatch stopWatch = new StopWatch(true);
hostname = socket.getInetAddress().getHostName();
final String requestId = UUID.randomUUID().toString();
logger.info("Received request {} from {}", requestId, hostname);
logger.debug("Received request {} from {}", requestId, hostname);
String requestorDn = getRequestorDN(socket);
@ -130,20 +130,23 @@ public class SocketProtocolListener extends SocketListener implements ProtocolLi
final ProtocolMessageUnmarshaller<ProtocolMessage> unmarshaller = protocolContext.createUnmarshaller();
final InputStream inStream = socket.getInputStream();
final CopyingInputStream copyingInputStream = new CopyingInputStream(inStream, maxMsgBuffer); // don't copy more than 1 MB
logger.debug("Request {} has a message length of {}", requestId, copyingInputStream.getNumberOfBytesCopied());
final ProtocolMessage request;
try {
request = unmarshaller.unmarshal(copyingInputStream);
} finally {
receivedMessage = copyingInputStream.getBytesRead();
if (logger.isDebugEnabled()) {
logger.debug("Received message: " + new String(receivedMessage));
}
}
request.setRequestorDN(requestorDn);
// dispatch message to handler
ProtocolHandler desiredHandler = null;
for (final ProtocolHandler handler : getHandlers()) {
final Collection<ProtocolHandler> handlers = getHandlers();
for (final ProtocolHandler handler : handlers) {
if (handler.canHandle(request)) {
desiredHandler = handler;
break;
@ -152,6 +155,7 @@ public class SocketProtocolListener extends SocketListener implements ProtocolLi
// if no handler found, throw exception; otherwise handle request
if (desiredHandler == null) {
logger.error("Received request of type {} but none of the following Protocol Handlers were able to process the request: {}", request.getType(), handlers);
throw new ProtocolException("No handler assigned to handle message type: " + request.getType());
} else {
final ProtocolMessage response = desiredHandler.handle(request);
@ -169,7 +173,8 @@ public class SocketProtocolListener extends SocketListener implements ProtocolLi
}
stopWatch.stop();
logger.info("Finished processing request {} (type={}, length={} bytes) in {} millis", requestId, request.getType(), receivedMessage.length, stopWatch.getDuration(TimeUnit.MILLISECONDS));
logger.info("Finished processing request {} (type={}, length={} bytes) from {} in {} millis",
requestId, request.getType(), receivedMessage.length, hostname, stopWatch.getDuration(TimeUnit.MILLISECONDS));
} catch (final IOException | ProtocolException e) {
logger.warn("Failed processing protocol message from " + hostname + " due to " + e, e);

View File

@ -24,17 +24,17 @@ import java.net.SocketException;
import java.util.Set;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.nifi.cluster.protocol.ClusterManagerProtocolSender;
import org.apache.nifi.cluster.protocol.ClusterCoordinationProtocolSender;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.cluster.protocol.ProtocolContext;
import org.apache.nifi.cluster.protocol.ProtocolException;
import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller;
import org.apache.nifi.cluster.protocol.ProtocolMessageUnmarshaller;
import org.apache.nifi.cluster.protocol.message.DisconnectMessage;
import org.apache.nifi.cluster.protocol.message.FlowRequestMessage;
import org.apache.nifi.cluster.protocol.message.FlowResponseMessage;
import org.apache.nifi.cluster.protocol.message.NodeStatusChangeMessage;
import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType;
@ -56,13 +56,13 @@ import org.apache.nifi.util.NiFiProperties;
* allowable time for communication with the node.
*
*/
public class ClusterManagerProtocolSenderImpl implements ClusterManagerProtocolSender {
public class StandardClusterCoordinationProtocolSender implements ClusterCoordinationProtocolSender {
private final ProtocolContext<ProtocolMessage> protocolContext;
private final SocketConfiguration socketConfiguration;
private int handshakeTimeoutSeconds;
public ClusterManagerProtocolSenderImpl(final SocketConfiguration socketConfiguration, final ProtocolContext<ProtocolMessage> protocolContext) {
public StandardClusterCoordinationProtocolSender(final SocketConfiguration socketConfiguration, final ProtocolContext<ProtocolMessage> protocolContext) {
if (socketConfiguration == null) {
throw new IllegalArgumentException("Socket configuration may not be null.");
} else if (protocolContext == null) {
@ -77,47 +77,6 @@ public class ClusterManagerProtocolSenderImpl implements ClusterManagerProtocolS
public void setBulletinRepository(final BulletinRepository bulletinRepository) {
}
/**
* Requests the data flow from a node.
*
* @param msg a message
* @return the message response
* @throws ProtocolException if the message failed to be sent or the
* response was malformed
*/
@Override
public FlowResponseMessage requestFlow(final FlowRequestMessage msg) throws ProtocolException {
Socket socket = null;
try {
socket = createSocket(msg.getNodeId(), false);
try {
// marshal message to output stream
final ProtocolMessageMarshaller<ProtocolMessage> marshaller = protocolContext.createMarshaller();
marshaller.marshal(msg, socket.getOutputStream());
} catch (final IOException ioe) {
throw new ProtocolException("Failed marshalling '" + msg.getType() + "' protocol message due to: " + ioe, ioe);
}
final ProtocolMessage response;
try {
// unmarshall response and return
final ProtocolMessageUnmarshaller<ProtocolMessage> unmarshaller = protocolContext.createUnmarshaller();
response = unmarshaller.unmarshal(socket.getInputStream());
} catch (final IOException ioe) {
throw new ProtocolException("Failed unmarshalling '" + MessageType.FLOW_RESPONSE + "' protocol message due to: " + ioe, ioe);
}
if (MessageType.FLOW_RESPONSE == response.getType()) {
return (FlowResponseMessage) response;
} else {
throw new ProtocolException("Expected message type '" + MessageType.FLOW_RESPONSE + "' but found '" + response.getType() + "'");
}
} finally {
SocketUtils.closeQuietly(socket);
}
}
/**
* Requests a node to reconnect to the cluster. The configured value for
@ -225,8 +184,12 @@ public class ClusterManagerProtocolSenderImpl implements ClusterManagerProtocolS
@Override
public void notifyNodeStatusChange(final Set<NodeIdentifier> nodesToNotify, final NodeStatusChangeMessage msg) {
if (nodesToNotify.isEmpty()) {
return;
}
final NiFiProperties properties = NiFiProperties.getInstance();
final int numThreads = Math.min(nodesToNotify.size(), properties.getClusterManagerProtocolThreads());
final int numThreads = Math.min(nodesToNotify.size(), properties.getClusterNodeProtocolThreads());
final byte[] msgBytes;
try (final ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
@ -237,7 +200,18 @@ public class ClusterManagerProtocolSenderImpl implements ClusterManagerProtocolS
throw new ProtocolException("Failed to marshal NodeStatusChangeMessage", e);
}
final ExecutorService executor = Executors.newFixedThreadPool(numThreads);
final ExecutorService executor = Executors.newFixedThreadPool(numThreads, new ThreadFactory() {
private final AtomicInteger counter = new AtomicInteger(0);
@Override
public Thread newThread(final Runnable r) {
final Thread thread = Executors.defaultThreadFactory().newThread(r);
thread.setDaemon(true);
thread.setName("Notify Cluster of Node Status Change-" + counter.incrementAndGet());
return thread;
}
});
for (final NodeIdentifier nodeId : nodesToNotify) {
executor.submit(new Runnable() {
@Override

View File

@ -16,32 +16,38 @@
*/
package org.apache.nifi.cluster.protocol.jaxb.message;
import java.util.List;
import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus;
import org.apache.nifi.cluster.protocol.ComponentRevision;
import org.apache.nifi.cluster.protocol.DataFlow;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.cluster.protocol.StandardDataFlow;
/**
*/
public class AdaptedConnectionResponse {
private StandardDataFlow dataFlow;
private DataFlow dataFlow;
private NodeIdentifier nodeIdentifier;
private String rejectionReason;
private int tryLaterSeconds;
private Integer managerRemoteInputPort;
private Boolean managerRemoteCommsSecure;
private String instanceId;
private List<NodeConnectionStatus> nodeStatuses;
private List<ComponentRevision> componentRevisions;
public AdaptedConnectionResponse() {
}
@XmlJavaTypeAdapter(DataFlowAdapter.class)
public StandardDataFlow getDataFlow() {
public DataFlow getDataFlow() {
return dataFlow;
}
public void setDataFlow(StandardDataFlow dataFlow) {
public void setDataFlow(DataFlow dataFlow) {
this.dataFlow = dataFlow;
}
@ -97,4 +103,20 @@ public class AdaptedConnectionResponse {
public String getInstanceId() {
return instanceId;
}
public void setNodeConnectionStatuses(List<NodeConnectionStatus> connectionStatuses) {
this.nodeStatuses = connectionStatuses;
}
public List<NodeConnectionStatus> getNodeConnectionStatuses() {
return this.nodeStatuses;
}
public List<ComponentRevision> getComponentRevisions() {
return componentRevisions;
}
public void setComponentRevisions(List<ComponentRevision> componentRevisions) {
this.componentRevisions = componentRevisions;
}
}

View File

@ -16,6 +16,8 @@
*/
package org.apache.nifi.cluster.protocol.jaxb.message;
import java.util.Set;
import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus;
@ -27,7 +29,7 @@ public class AdaptedHeartbeat {
private NodeIdentifier nodeIdentifier;
private byte[] payload;
private boolean primary;
private Set<String> roles;
private NodeConnectionStatus connectionStatus;
public AdaptedHeartbeat() {
@ -42,12 +44,12 @@ public class AdaptedHeartbeat {
this.nodeIdentifier = nodeIdentifier;
}
public boolean isPrimary() {
return primary;
public Set<String> getRoles() {
return roles;
}
public void setPrimary(boolean primary) {
this.primary = primary;
public void setRoles(Set<String> roles) {
this.roles = roles;
}
public void setConnectionStatus(NodeConnectionStatus connectionStatus) {

View File

@ -17,14 +17,36 @@
package org.apache.nifi.cluster.protocol.jaxb.message;
import java.util.Set;
import org.apache.nifi.cluster.coordination.node.DisconnectionCode;
import org.apache.nifi.cluster.coordination.node.NodeConnectionState;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
public class AdaptedNodeConnectionStatus {
private Long updateId;
private NodeIdentifier nodeId;
private NodeConnectionState state;
private DisconnectionCode disconnectCode;
private String disconnectReason;
private Long connectionRequestTime;
private Set<String> roles;
public Long getUpdateId() {
return updateId;
}
public void setUpdateId(Long updateId) {
this.updateId = updateId;
}
public NodeIdentifier getNodeId() {
return nodeId;
}
public void setNodeId(NodeIdentifier nodeId) {
this.nodeId = nodeId;
}
public NodeConnectionState getState() {
return state;
@ -57,4 +79,12 @@ public class AdaptedNodeConnectionStatus {
public void setConnectionRequestTime(Long connectionRequestTime) {
this.connectionRequestTime = connectionRequestTime;
}
public Set<String> getRoles() {
return roles;
}
public void setRoles(Set<String> roles) {
this.roles = roles;
}
}

View File

@ -34,6 +34,8 @@ public class ConnectionResponseAdapter extends XmlAdapter<AdaptedConnectionRespo
aCr.setManagerRemoteInputPort(cr.getManagerRemoteInputPort());
aCr.setManagerRemoteCommsSecure(cr.isManagerRemoteCommsSecure());
aCr.setInstanceId(cr.getInstanceId());
aCr.setNodeConnectionStatuses(cr.getNodeConnectionStatuses());
aCr.setComponentRevisions(cr.getComponentRevisions());
}
return aCr;
}
@ -46,7 +48,8 @@ public class ConnectionResponseAdapter extends XmlAdapter<AdaptedConnectionRespo
return ConnectionResponse.createRejectionResponse(aCr.getRejectionReason());
} else {
return new ConnectionResponse(aCr.getNodeIdentifier(), aCr.getDataFlow(),
aCr.getManagerRemoteInputPort(), aCr.isManagerRemoteCommsSecure(), aCr.getInstanceId());
aCr.getManagerRemoteInputPort(), aCr.isManagerRemoteCommsSecure(),
aCr.getInstanceId(), aCr.getNodeConnectionStatuses(), aCr.getComponentRevisions());
}
}

View File

@ -18,15 +18,15 @@ package org.apache.nifi.cluster.protocol.jaxb.message;
import javax.xml.bind.annotation.adapters.XmlAdapter;
import org.apache.nifi.cluster.protocol.DataFlow;
import org.apache.nifi.cluster.protocol.StandardDataFlow;
/**
*/
public class DataFlowAdapter extends XmlAdapter<AdaptedDataFlow, StandardDataFlow> {
public class DataFlowAdapter extends XmlAdapter<AdaptedDataFlow, DataFlow> {
@Override
public AdaptedDataFlow marshal(final StandardDataFlow df) {
public AdaptedDataFlow marshal(final DataFlow df) {
final AdaptedDataFlow aDf = new AdaptedDataFlow();
if (df != null) {
@ -39,7 +39,7 @@ public class DataFlowAdapter extends XmlAdapter<AdaptedDataFlow, StandardDataFlo
}
@Override
public StandardDataFlow unmarshal(final AdaptedDataFlow aDf) {
public DataFlow unmarshal(final AdaptedDataFlow aDf) {
final StandardDataFlow dataFlow = new StandardDataFlow(aDf.getFlow(), aDf.getSnippets());
dataFlow.setAutoStartProcessors(aDf.isAutoStartProcessors());
return dataFlow;

View File

@ -35,7 +35,7 @@ public class HeartbeatAdapter extends XmlAdapter<AdaptedHeartbeat, Heartbeat> {
aHb.setPayload(hb.getPayload());
// set leader flag
aHb.setPrimary(hb.isPrimary());
aHb.setRoles(hb.getRoles());
// set connected flag
aHb.setConnectionStatus(hb.getConnectionStatus());
@ -46,7 +46,7 @@ public class HeartbeatAdapter extends XmlAdapter<AdaptedHeartbeat, Heartbeat> {
@Override
public Heartbeat unmarshal(final AdaptedHeartbeat aHb) {
return new Heartbeat(aHb.getNodeIdentifier(), aHb.isPrimary(), aHb.getConnectionStatus(), aHb.getPayload());
return new Heartbeat(aHb.getNodeIdentifier(), aHb.getRoles(), aHb.getConnectionStatus(), aHb.getPayload());
}
}

View File

@ -37,5 +37,4 @@ public final class JaxbProtocolUtils {
throw new RuntimeException("Unable to create JAXBContext.", e);
}
}
}

View File

@ -25,16 +25,25 @@ public class NodeConnectionStatusAdapter extends XmlAdapter<AdaptedNodeConnectio
@Override
public NodeConnectionStatus unmarshal(final AdaptedNodeConnectionStatus adapted) throws Exception {
return new NodeConnectionStatus(adapted.getState(), adapted.getDisconnectCode(), adapted.getDisconnectReason(), adapted.getConnectionRequestTime());
return new NodeConnectionStatus(adapted.getUpdateId(),
adapted.getNodeId(),
adapted.getState(),
adapted.getDisconnectCode(),
adapted.getDisconnectReason(),
adapted.getConnectionRequestTime(),
adapted.getRoles());
}
@Override
public AdaptedNodeConnectionStatus marshal(final NodeConnectionStatus toAdapt) throws Exception {
final AdaptedNodeConnectionStatus adapted = new AdaptedNodeConnectionStatus();
adapted.setUpdateId(toAdapt.getUpdateIdentifier());
adapted.setNodeId(toAdapt.getNodeIdentifier());
adapted.setConnectionRequestTime(toAdapt.getConnectionRequestTime());
adapted.setDisconnectCode(toAdapt.getDisconnectCode());
adapted.setDisconnectReason(toAdapt.getDisconnectReason());
adapted.setState(toAdapt.getState());
adapted.setRoles(toAdapt.getRoles());
return adapted;
}
}

View File

@ -16,14 +16,15 @@
*/
package org.apache.nifi.cluster.protocol.message;
import javax.xml.bind.annotation.XmlRootElement;
import org.apache.nifi.cluster.protocol.ConnectionResponse;
import javax.xml.bind.annotation.XmlRootElement;
@XmlRootElement(name = "connectionResponseMessage")
public class ConnectionResponseMessage extends ProtocolMessage {
private ConnectionResponse connectionResponse;
private String clusterManagerDN;
private String coordinatorDN;
public ConnectionResponseMessage() {
}
@ -35,24 +36,24 @@ public class ConnectionResponseMessage extends ProtocolMessage {
public void setConnectionResponse(final ConnectionResponse connectionResponse) {
this.connectionResponse = connectionResponse;
if (clusterManagerDN != null) {
this.connectionResponse.setClusterManagerDN(clusterManagerDN);
if (coordinatorDN != null) {
this.connectionResponse.setCoordinatorDN(coordinatorDN);
}
}
public void setClusterManagerDN(final String dn) {
public void setCoordinatorDN(final String dn) {
if (connectionResponse != null) {
connectionResponse.setClusterManagerDN(dn);
connectionResponse.setCoordinatorDN(dn);
}
this.clusterManagerDN = dn;
this.coordinatorDN = dn;
}
/**
* @return the DN of the NCM, if it is available or <code>null</code>
* @return the DN of the Coordinator, if it is available or <code>null</code>
* otherwise
*/
public String getClusterManagerDN() {
return clusterManagerDN;
public String getCoordinatorDN() {
return coordinatorDN;
}
@Override

View File

@ -14,36 +14,34 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.protocol.message;
import javax.xml.bind.annotation.XmlRootElement;
import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.cluster.protocol.jaxb.message.NodeConnectionStatusAdapter;
import org.apache.nifi.cluster.protocol.jaxb.message.NodeIdentifierAdapter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Message to indicate that the status of a node in the cluster has changed
*/
@XmlRootElement(name = "nodeStatusChange")
public class NodeStatusChangeMessage extends ProtocolMessage {
private static final Logger logger = LoggerFactory.getLogger(NodeStatusChangeMessage.class);
private NodeConnectionStatus connectionStatus;
private NodeIdentifier nodeId;
private Long statusUpdateId = -1L;
@Override
public MessageType getType() {
return MessageType.NODE_STATUS_CHANGE;
}
public void setNodeConnectionStatus(final NodeConnectionStatus status) {
this.connectionStatus = status;
}
public NodeConnectionStatus getNodeConnectionStatus() {
return connectionStatus;
}
@XmlJavaTypeAdapter(NodeIdentifierAdapter.class)
public NodeIdentifier getNodeId() {
return nodeId;
}
@ -52,11 +50,17 @@ public class NodeStatusChangeMessage extends ProtocolMessage {
this.nodeId = nodeId;
}
public Long getStatusUpdateIdentifier() {
return statusUpdateId;
@XmlJavaTypeAdapter(NodeConnectionStatusAdapter.class)
public NodeConnectionStatus getNodeConnectionStatus() {
return connectionStatus;
}
public void setStatusUpdateIdentifier(Long statusUpdateId) {
this.statusUpdateId = statusUpdateId;
public void setNodeConnectionStatus(final NodeConnectionStatus status) {
this.connectionStatus = status;
}
@Override
public String toString() {
return "NodeStatusChangeMessage[nodeId=" + nodeId + ", status=" + getNodeConnectionStatus() + "]";
}
}

View File

@ -16,9 +16,13 @@
*/
package org.apache.nifi.cluster.protocol.message;
import java.util.List;
import javax.xml.bind.annotation.XmlRootElement;
import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus;
import org.apache.nifi.cluster.protocol.ComponentRevision;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.cluster.protocol.StandardDataFlow;
import org.apache.nifi.cluster.protocol.jaxb.message.NodeIdentifierAdapter;
@ -34,6 +38,8 @@ public class ReconnectionRequestMessage extends ProtocolMessage {
private Integer managerRemoteSiteListeningPort;
private Boolean managerRemoteSiteCommsSecure;
private String instanceId;
private List<NodeConnectionStatus> nodeStatuses;
private List<ComponentRevision> componentRevisions;
public ReconnectionRequestMessage() {
}
@ -91,4 +97,20 @@ public class ReconnectionRequestMessage extends ProtocolMessage {
public String getInstanceId() {
return instanceId;
}
public void setNodeConnectionStatuses(List<NodeConnectionStatus> statuses) {
this.nodeStatuses = statuses;
}
public List<NodeConnectionStatus> getNodeConnectionStatuses() {
return nodeStatuses;
}
public List<ComponentRevision> getComponentRevisions() {
return componentRevisions;
}
public void setComponentRevisions(List<ComponentRevision> componentRevisions) {
this.componentRevisions = componentRevisions;
}
}

View File

@ -1,60 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.protocol.spring;
import java.util.concurrent.TimeUnit;
import org.apache.nifi.io.socket.multicast.MulticastConfiguration;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.util.NiFiProperties;
import org.springframework.beans.factory.FactoryBean;
/**
* Factory bean for creating a singleton MulticastConfiguration instance.
*/
public class MulticastConfigurationFactoryBean implements FactoryBean {
private MulticastConfiguration configuration;
private NiFiProperties properties;
@Override
public Object getObject() throws Exception {
if (configuration == null) {
configuration = new MulticastConfiguration();
final int timeout = (int) FormatUtils.getTimeDuration(properties.getClusterProtocolSocketTimeout(), TimeUnit.MILLISECONDS);
configuration.setSocketTimeout(timeout);
configuration.setReuseAddress(true);
}
return configuration;
}
@Override
public Class getObjectType() {
return MulticastConfiguration.class;
}
@Override
public boolean isSingleton() {
return true;
}
public void setProperties(NiFiProperties properties) {
this.properties = properties;
}
}

View File

@ -16,15 +16,14 @@
*/
package org.apache.nifi.cluster.protocol.spring;
import java.util.concurrent.TimeUnit;
import org.apache.nifi.io.socket.SSLContextFactory;
import org.apache.nifi.io.socket.ServerSocketConfiguration;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.util.NiFiProperties;
import org.springframework.beans.factory.FactoryBean;
import java.util.concurrent.TimeUnit;
/**
* Factory bean for creating a singleton ServerSocketConfiguration instance.
*/
@ -39,7 +38,7 @@ public class ServerSocketConfigurationFactoryBean implements FactoryBean<ServerS
configuration = new ServerSocketConfiguration();
configuration.setNeedClientAuth(properties.getNeedClientAuth());
final int timeout = (int) FormatUtils.getTimeDuration(properties.getClusterProtocolSocketTimeout(), TimeUnit.MILLISECONDS);
final int timeout = (int) FormatUtils.getTimeDuration(properties.getClusterNodeReadTimeout(), TimeUnit.MILLISECONDS);
configuration.setSocketTimeout(timeout);
configuration.setReuseAddress(true);
if (Boolean.valueOf(properties.getProperty(NiFiProperties.CLUSTER_PROTOCOL_IS_SECURE))) {

View File

@ -16,15 +16,14 @@
*/
package org.apache.nifi.cluster.protocol.spring;
import java.util.concurrent.TimeUnit;
import org.apache.nifi.io.socket.SSLContextFactory;
import org.apache.nifi.io.socket.SocketConfiguration;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.util.NiFiProperties;
import org.springframework.beans.factory.FactoryBean;
import java.util.concurrent.TimeUnit;
/**
* Factory bean for creating a singleton SocketConfiguration instance.
*/
@ -39,7 +38,7 @@ public class SocketConfigurationFactoryBean implements FactoryBean<SocketConfigu
if (configuration == null) {
configuration = new SocketConfiguration();
final int timeout = (int) FormatUtils.getTimeDuration(properties.getClusterProtocolSocketTimeout(), TimeUnit.MILLISECONDS);
final int timeout = (int) FormatUtils.getTimeDuration(properties.getClusterNodeReadTimeout(), TimeUnit.MILLISECONDS);
configuration.setSocketTimeout(timeout);
configuration.setReuseAddress(true);
if (Boolean.valueOf(properties.getProperty(NiFiProperties.CLUSTER_PROTOCOL_IS_SECURE))) {

View File

@ -38,47 +38,30 @@
<property name="properties" ref="nifiProperties"/>
</bean>
<!-- multicast configuration -->
<bean id="protocolMulticastConfiguration" class="org.apache.nifi.cluster.protocol.spring.MulticastConfigurationFactoryBean">
<property name="properties" ref="nifiProperties"/>
</bean>
<!-- cluster manager protocol sender -->
<bean id="clusterManagerProtocolSender" class="org.apache.nifi.cluster.protocol.impl.ClusterManagerProtocolSenderImpl">
<bean id="clusterCoordinationProtocolSender" class="org.apache.nifi.cluster.protocol.impl.StandardClusterCoordinationProtocolSender">
<constructor-arg ref="protocolSocketConfiguration"/>
<constructor-arg ref="protocolContext"/>
<property name="handshakeTimeout">
<bean factory-bean="nifiProperties" factory-method="getClusterProtocolConnectionHandshakeTimeout"/>
<bean factory-bean="nifiProperties" factory-method="getClusterNodeConnectionTimeout"/>
</property>
</bean>
<!-- cluster manager protocol listener -->
<bean id="clusterManagerProtocolListener" class="org.apache.nifi.cluster.protocol.impl.SocketProtocolListener">
<constructor-arg index="0">
<bean factory-bean="nifiProperties" factory-method="getClusterManagerProtocolThreads"/>
</constructor-arg>
<constructor-arg index="1">
<bean factory-bean="nifiProperties" factory-method="getClusterManagerProtocolPort"/>
</constructor-arg>
<constructor-arg ref="protocolServerSocketConfiguration" index="2"/>
<constructor-arg ref="protocolContext" index="3"/>
</bean>
<!-- cluster manager sender/listener -->
<bean id="clusterManagerProtocolSenderListener" class="org.apache.nifi.cluster.protocol.impl.ClusterManagerProtocolSenderListener">
<constructor-arg ref="clusterManagerProtocolSender"/>
<constructor-arg ref="clusterManagerProtocolListener"/>
<bean id="clusterCoordinationProtocolSenderListener" class="org.apache.nifi.cluster.protocol.impl.ClusterCoordinationProtocolSenderListener">
<constructor-arg ref="clusterCoordinationProtocolSender"/>
<constructor-arg ref="protocolListener"/>
</bean>
<!-- node protocol sender -->
<bean id="nodeProtocolSender" class="org.apache.nifi.cluster.protocol.impl.NodeProtocolSenderImpl">
<constructor-arg ref="clusterManagerProtocolServiceLocator"/>
<bean id="nodeProtocolSender" class="org.apache.nifi.cluster.coordination.node.CuratorNodeProtocolSender">
<constructor-arg ref="protocolSocketConfiguration"/>
<constructor-arg ref="protocolContext"/>
<constructor-arg ref="nifiProperties"/>
</bean>
<!-- node protocol listener -->
<bean id="nodeProtocolListener" class="org.apache.nifi.cluster.protocol.impl.SocketProtocolListener">
<!-- protocol listener -->
<bean id="protocolListener" class="org.apache.nifi.cluster.protocol.impl.SocketProtocolListener">
<constructor-arg index="0">
<bean factory-bean="nifiProperties" factory-method="getClusterNodeProtocolThreads"/>
</constructor-arg>
@ -92,19 +75,6 @@
<!-- node sender/listener -->
<bean id="nodeProtocolSenderListener" class="org.apache.nifi.cluster.protocol.impl.NodeProtocolSenderListener">
<constructor-arg ref="nodeProtocolSender"/>
<constructor-arg ref="nodeProtocolListener"/>
<constructor-arg ref="protocolListener"/>
</bean>
<!-- cluster services broadcaster -->
<bean id="clusterServicesBroadcaster" class="org.apache.nifi.cluster.protocol.impl.ClusterServicesBroadcaster">
<constructor-arg index="0">
<bean factory-bean="nifiProperties" factory-method="getClusterProtocolMulticastAddress"/>
</constructor-arg>
<constructor-arg ref="protocolMulticastConfiguration" index="1"/>
<constructor-arg ref="protocolContext" index="2"/>
<constructor-arg index="3">
<bean factory-bean="nifiProperties" factory-method="getClusterProtocolMulticastServiceBroadcastDelay"/>
</constructor-arg>
</bean>
</beans>

View File

@ -1,133 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.protocol.impl;
import java.io.IOException;
import java.net.InetAddress;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.cluster.protocol.ProtocolContext;
import org.apache.nifi.cluster.protocol.ProtocolException;
import org.apache.nifi.cluster.protocol.ProtocolHandler;
import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext;
import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils;
import org.apache.nifi.cluster.protocol.message.FlowRequestMessage;
import org.apache.nifi.cluster.protocol.message.FlowResponseMessage;
import org.apache.nifi.cluster.protocol.message.PingMessage;
import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
import org.apache.nifi.io.socket.ServerSocketConfiguration;
import org.apache.nifi.io.socket.SocketConfiguration;
import org.junit.After;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.fail;
import org.junit.Before;
import org.junit.Test;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
public class ClusterManagerProtocolSenderImplTest {
private InetAddress address;
private int port;
private SocketProtocolListener listener;
private ClusterManagerProtocolSenderImpl sender;
private ProtocolHandler mockHandler;
@Before
public void setup() throws IOException {
address = InetAddress.getLocalHost();
ServerSocketConfiguration serverSocketConfiguration = new ServerSocketConfiguration();
serverSocketConfiguration.setSocketTimeout(2000);
mockHandler = mock(ProtocolHandler.class);
ProtocolContext protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT);
listener = new SocketProtocolListener(5, 0, serverSocketConfiguration, protocolContext);
listener.addHandler(mockHandler);
listener.start();
port = listener.getPort();
SocketConfiguration socketConfiguration = new SocketConfiguration();
sender = new ClusterManagerProtocolSenderImpl(socketConfiguration, protocolContext);
}
@After
public void teardown() throws IOException {
if (listener.isRunning()) {
listener.stop();
}
}
@Test
public void testRequestFlow() throws Exception {
when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(new FlowResponseMessage());
FlowRequestMessage request = new FlowRequestMessage();
request.setNodeId(new NodeIdentifier("id", "api-address", 1, "localhost", port, "localhost", 3821, false));
FlowResponseMessage response = sender.requestFlow(request);
assertNotNull(response);
}
@Test
public void testRequestFlowWithBadResponseMessage() throws Exception {
when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(new PingMessage());
FlowRequestMessage request = new FlowRequestMessage();
request.setNodeId(new NodeIdentifier("id", "api-address", 1, "localhost", port, "localhost", 3821, false));
try {
sender.requestFlow(request);
fail("failed to throw exception");
} catch (ProtocolException pe) {
}
}
@Test
public void testRequestFlowDelayedResponse() throws Exception {
final int time = 250;
sender.getSocketConfiguration().setSocketTimeout(time);
when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
when(mockHandler.handle(any(ProtocolMessage.class))).thenAnswer(new Answer<FlowResponseMessage>() {
@Override
public FlowResponseMessage answer(InvocationOnMock invocation) throws Throwable {
Thread.sleep(time * 3);
return new FlowResponseMessage();
}
});
FlowRequestMessage request = new FlowRequestMessage();
request.setNodeId(new NodeIdentifier("id", "api-address", 1, "localhost", port, "localhost", 3821, false));
try {
sender.requestFlow(request);
fail("failed to throw exception");
} catch (ProtocolException pe) {
}
}
}

View File

@ -1,134 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.protocol.impl;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.net.DatagramPacket;
import java.net.InetSocketAddress;
import java.net.MulticastSocket;
import org.apache.nifi.cluster.protocol.ProtocolContext;
import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller;
import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext;
import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils;
import org.apache.nifi.cluster.protocol.message.PingMessage;
import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
import org.apache.nifi.cluster.protocol.message.ServiceBroadcastMessage;
import org.apache.nifi.io.socket.multicast.MulticastConfiguration;
import org.apache.nifi.io.socket.multicast.MulticastUtils;
import org.junit.After;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import org.junit.Before;
import org.junit.Ignore;
import org.junit.Test;
/**
*/
public class ClusterServiceDiscoveryTest {
private ClusterServiceDiscovery discovery;
private String serviceName;
private MulticastSocket socket;
private InetSocketAddress multicastAddress;
private MulticastConfiguration configuration;
private ProtocolContext protocolContext;
@Before
public void setup() throws Exception {
serviceName = "some-service";
multicastAddress = new InetSocketAddress("225.1.1.1", 22222);
configuration = new MulticastConfiguration();
protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT);
discovery = new ClusterServiceDiscovery(serviceName, multicastAddress, configuration, protocolContext);
discovery.start();
socket = MulticastUtils.createMulticastSocket(multicastAddress.getPort(), configuration);
}
@After
public void teardown() throws IOException {
try {
if (discovery.isRunning()) {
discovery.stop();
}
} finally {
MulticastUtils.closeQuietly(socket);
}
}
@Ignore("Test needs to be fixed. Requires an active network connection")
@Test
public void testGetAddressOnStartup() {
assertNull(discovery.getService());
}
@Ignore("This test has an NPE after ignoring another...perhaps has a bad inter-test dependency")
@Test
public void testGetAddressAfterBroadcast() throws Exception {
ServiceBroadcastMessage msg = new ServiceBroadcastMessage();
msg.setServiceName("some-service");
msg.setAddress("3.3.3.3");
msg.setPort(1234);
// marshal message to output stream
ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller();
ByteArrayOutputStream baos = new ByteArrayOutputStream();
marshaller.marshal(msg, baos);
byte[] requestPacketBytes = baos.toByteArray();
DatagramPacket packet = new DatagramPacket(requestPacketBytes, requestPacketBytes.length, multicastAddress);
socket.send(packet);
Thread.sleep(250);
InetSocketAddress updatedAddress = discovery.getService().getServiceAddress();
assertEquals("some-service", discovery.getServiceName());
assertEquals("3.3.3.3", updatedAddress.getHostName());
assertEquals(1234, updatedAddress.getPort());
}
@Ignore("Test needs to be fixed. Requires an active network connection")
@Test
public void testBadBroadcastMessage() throws Exception {
ProtocolMessage msg = new PingMessage();
// marshal message to output stream
ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller();
ByteArrayOutputStream baos = new ByteArrayOutputStream();
marshaller.marshal(msg, baos);
byte[] requestPacketBytes = baos.toByteArray();
DatagramPacket packet = new DatagramPacket(requestPacketBytes, requestPacketBytes.length, multicastAddress);
socket.send(packet);
Thread.sleep(250);
assertNull(discovery.getService());
}
}

View File

@ -1,121 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.protocol.impl;
import java.net.InetSocketAddress;
import java.util.concurrent.TimeUnit;
import org.apache.nifi.io.socket.multicast.DiscoverableService;
import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import org.junit.Before;
import org.junit.Test;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import org.mockito.stubbing.OngoingStubbing;
public class ClusterServiceLocatorTest {
private ClusterServiceDiscovery mockServiceDiscovery;
private int fixedPort;
private DiscoverableService fixedService;
private ClusterServiceLocator serviceDiscoveryLocator;
private ClusterServiceLocator serviceDiscoveryFixedPortLocator;
private ClusterServiceLocator fixedServiceLocator;
@Before
public void setup() throws Exception {
fixedPort = 1;
mockServiceDiscovery = mock(ClusterServiceDiscovery.class);
fixedService = new DiscoverableServiceImpl("some-service", InetSocketAddress.createUnresolved("some-host", 20));
serviceDiscoveryLocator = new ClusterServiceLocator(mockServiceDiscovery);
serviceDiscoveryFixedPortLocator = new ClusterServiceLocator(mockServiceDiscovery, fixedPort);
fixedServiceLocator = new ClusterServiceLocator(fixedService);
}
@Test
public void getServiceWhenServiceDiscoveryNotStarted() {
assertNull(serviceDiscoveryLocator.getService());
}
@Test
public void getServiceWhenServiceDiscoveryFixedPortNotStarted() {
assertNull(serviceDiscoveryLocator.getService());
}
@Test
public void getServiceWhenFixedServiceNotStarted() {
assertEquals(fixedService, fixedServiceLocator.getService());
}
@Test
public void getServiceNotOnFirstAttempt() {
ClusterServiceLocator.AttemptsConfig config = new ClusterServiceLocator.AttemptsConfig();
config.setNumAttempts(2);
config.setTimeBetweenAttempsUnit(TimeUnit.SECONDS);
config.setTimeBetweenAttempts(1);
serviceDiscoveryLocator.setAttemptsConfig(config);
OngoingStubbing<DiscoverableService> stubbing = null;
for (int i = 0; i < config.getNumAttempts() - 1; i++) {
if (stubbing == null) {
stubbing = when(mockServiceDiscovery.getService()).thenReturn(null);
} else {
stubbing.thenReturn(null);
}
}
stubbing.thenReturn(fixedService);
assertEquals(fixedService, serviceDiscoveryLocator.getService());
}
@Test
public void getServiceNotOnFirstAttemptWithFixedPort() {
ClusterServiceLocator.AttemptsConfig config = new ClusterServiceLocator.AttemptsConfig();
config.setNumAttempts(2);
config.setTimeBetweenAttempsUnit(TimeUnit.SECONDS);
config.setTimeBetweenAttempts(1);
serviceDiscoveryFixedPortLocator.setAttemptsConfig(config);
OngoingStubbing<DiscoverableService> stubbing = null;
for (int i = 0; i < config.getNumAttempts() - 1; i++) {
if (stubbing == null) {
stubbing = when(mockServiceDiscovery.getService()).thenReturn(null);
} else {
stubbing.thenReturn(null);
}
}
stubbing.thenReturn(fixedService);
InetSocketAddress resultAddress = InetSocketAddress.createUnresolved(fixedService.getServiceAddress().getHostName(), fixedPort);
DiscoverableService resultService = new DiscoverableServiceImpl(fixedService.getServiceName(), resultAddress);
assertEquals(resultService, serviceDiscoveryFixedPortLocator.getService());
}
}

View File

@ -1,131 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.protocol.impl;
import java.net.InetSocketAddress;
import org.apache.nifi.cluster.protocol.ProtocolContext;
import org.apache.nifi.cluster.protocol.ProtocolException;
import org.apache.nifi.cluster.protocol.ProtocolHandler;
import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext;
import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils;
import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
import org.apache.nifi.cluster.protocol.message.ServiceBroadcastMessage;
import org.apache.nifi.io.socket.multicast.DiscoverableService;
import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl;
import org.apache.nifi.io.socket.multicast.MulticastConfiguration;
import org.junit.After;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import org.junit.Before;
import org.junit.Ignore;
import org.junit.Test;
/**
*/
public class ClusterServicesBroadcasterTest {
private ClusterServicesBroadcaster broadcaster;
private MulticastProtocolListener listener;
private DummyProtocolHandler handler;
private InetSocketAddress multicastAddress;
private DiscoverableService broadcastedService;
private ProtocolContext protocolContext;
private MulticastConfiguration configuration;
@Before
public void setup() throws Exception {
broadcastedService = new DiscoverableServiceImpl("some-service", new InetSocketAddress("localhost", 11111));
multicastAddress = new InetSocketAddress("225.1.1.1", 22222);
configuration = new MulticastConfiguration();
protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT);
broadcaster = new ClusterServicesBroadcaster(multicastAddress, configuration, protocolContext, "500 ms");
broadcaster.addService(broadcastedService);
handler = new DummyProtocolHandler();
listener = new MulticastProtocolListener(5, multicastAddress, configuration, protocolContext);
listener.addHandler(handler);
}
@After
public void teardown() {
if (broadcaster.isRunning()) {
broadcaster.stop();
}
try {
if (listener.isRunning()) {
listener.stop();
}
} catch (Exception ex) {
ex.printStackTrace(System.out);
}
}
@Ignore("fails needs to be fixed")
@Test
public void testBroadcastReceived() throws Exception {
broadcaster.start();
listener.start();
Thread.sleep(1000);
listener.stop();
assertNotNull(handler.getProtocolMessage());
assertEquals(ProtocolMessage.MessageType.SERVICE_BROADCAST, handler.getProtocolMessage().getType());
final ServiceBroadcastMessage msg = (ServiceBroadcastMessage) handler.getProtocolMessage();
assertEquals(broadcastedService.getServiceName(), msg.getServiceName());
assertEquals(broadcastedService.getServiceAddress().getHostName(), msg.getAddress());
assertEquals(broadcastedService.getServiceAddress().getPort(), msg.getPort());
}
private class DummyProtocolHandler implements ProtocolHandler {
private ProtocolMessage protocolMessage;
@Override
public boolean canHandle(ProtocolMessage msg) {
return true;
}
@Override
public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException {
this.protocolMessage = msg;
return null;
}
public ProtocolMessage getProtocolMessage() {
return protocolMessage;
}
}
}

View File

@ -1,168 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.protocol.impl;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.net.DatagramPacket;
import java.net.InetSocketAddress;
import java.net.MulticastSocket;
import java.util.ArrayList;
import java.util.List;
import org.apache.nifi.cluster.protocol.ProtocolContext;
import org.apache.nifi.cluster.protocol.ProtocolException;
import org.apache.nifi.cluster.protocol.ProtocolHandler;
import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller;
import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext;
import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils;
import org.apache.nifi.cluster.protocol.message.MulticastProtocolMessage;
import org.apache.nifi.cluster.protocol.message.PingMessage;
import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
import org.apache.nifi.io.socket.multicast.MulticastConfiguration;
import org.apache.nifi.io.socket.multicast.MulticastUtils;
import org.junit.After;
import static org.junit.Assert.assertEquals;
import org.junit.Before;
import org.junit.Ignore;
import org.junit.Test;
public class MulticastProtocolListenerTest {
private MulticastProtocolListener listener;
private MulticastSocket socket;
private InetSocketAddress address;
private MulticastConfiguration configuration;
private ProtocolContext protocolContext;
@Before
public void setup() throws Exception {
address = new InetSocketAddress("226.1.1.1", 60000);
configuration = new MulticastConfiguration();
protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT);
listener = new MulticastProtocolListener(5, address, configuration, protocolContext);
listener.start();
socket = MulticastUtils.createMulticastSocket(address.getPort(), configuration);
}
@After
public void teardown() throws IOException {
try {
if (listener.isRunning()) {
listener.stop();
}
} finally {
MulticastUtils.closeQuietly(socket);
}
}
@Ignore("Test needs to be reworked. Fails if on a system without actiev network connection")
@Test
public void testBadRequest() throws Exception {
DelayedProtocolHandler handler = new DelayedProtocolHandler(0);
listener.addHandler(handler);
DatagramPacket packet = new DatagramPacket(new byte[]{5}, 1, address);
socket.send(packet);
Thread.sleep(250);
assertEquals(0, handler.getMessages().size());
}
@Ignore("this test works sometimes and fails others - needs work to be reliable")
@Test
public void testRequest() throws Exception {
ReflexiveProtocolHandler handler = new ReflexiveProtocolHandler();
listener.addHandler(handler);
ProtocolMessage msg = new PingMessage();
MulticastProtocolMessage multicastMsg = new MulticastProtocolMessage("some-id", msg);
// marshal message to output stream
ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller();
ByteArrayOutputStream baos = new ByteArrayOutputStream();
marshaller.marshal(multicastMsg, baos);
byte[] requestPacketBytes = baos.toByteArray();
DatagramPacket packet = new DatagramPacket(requestPacketBytes, requestPacketBytes.length, address);
socket.send(packet);
Thread.sleep(250);
assertEquals(1, handler.getMessages().size());
assertEquals(msg.getType(), handler.getMessages().get(0).getType());
}
private class ReflexiveProtocolHandler implements ProtocolHandler {
private List<ProtocolMessage> messages = new ArrayList<>();
@Override
public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException {
messages.add(msg);
return msg;
}
@Override
public boolean canHandle(ProtocolMessage msg) {
return true;
}
public List<ProtocolMessage> getMessages() {
return messages;
}
}
private class DelayedProtocolHandler implements ProtocolHandler {
private int delay = 0;
private List<ProtocolMessage> messages = new ArrayList<>();
public DelayedProtocolHandler(int delay) {
this.delay = delay;
}
@Override
public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException {
try {
messages.add(msg);
Thread.sleep(delay);
return null;
} catch (final InterruptedException ie) {
throw new ProtocolException(ie);
}
}
@Override
public boolean canHandle(ProtocolMessage msg) {
return true;
}
public List<ProtocolMessage> getMessages() {
return messages;
}
}
}

View File

@ -1,168 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.protocol.impl;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.UUID;
import org.apache.nifi.cluster.protocol.ConnectionRequest;
import org.apache.nifi.cluster.protocol.ConnectionResponse;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.cluster.protocol.ProtocolContext;
import org.apache.nifi.cluster.protocol.ProtocolException;
import org.apache.nifi.cluster.protocol.ProtocolHandler;
import org.apache.nifi.cluster.protocol.StandardDataFlow;
import org.apache.nifi.cluster.protocol.UnknownServiceAddressException;
import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext;
import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils;
import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage;
import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage;
import org.apache.nifi.cluster.protocol.message.PingMessage;
import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
import org.apache.nifi.io.socket.ServerSocketConfiguration;
import org.apache.nifi.io.socket.SocketConfiguration;
import org.apache.nifi.io.socket.multicast.DiscoverableService;
import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl;
import org.junit.After;
import org.junit.Before;
import org.junit.Ignore;
import org.junit.Test;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
@Ignore("Randomly tests... probably timing-specific")
public class NodeProtocolSenderImplTest {
private SocketProtocolListener listener;
private NodeProtocolSenderImpl sender;
private DiscoverableService service;
private ServerSocketConfiguration serverSocketConfiguration;
private ClusterServiceLocator mockServiceLocator;
private ProtocolHandler mockHandler;
private NodeIdentifier nodeIdentifier;
@Before
public void setup() throws IOException {
serverSocketConfiguration = new ServerSocketConfiguration();
mockServiceLocator = mock(ClusterServiceLocator.class);
mockHandler = mock(ProtocolHandler.class);
nodeIdentifier = new NodeIdentifier("1", "localhost", 1234, "localhost", 5678, "localhost", 3821, false);
ProtocolContext protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT);
listener = new SocketProtocolListener(5, 0, serverSocketConfiguration, protocolContext);
listener.setShutdownListenerSeconds(3);
listener.addHandler(mockHandler);
listener.start();
service = new DiscoverableServiceImpl("some-service", new InetSocketAddress("localhost", listener.getPort()));
SocketConfiguration socketConfiguration = new SocketConfiguration();
socketConfiguration.setReuseAddress(true);
sender = new NodeProtocolSenderImpl(mockServiceLocator, socketConfiguration, protocolContext);
}
@After
public void teardown() throws IOException {
if (listener.isRunning()) {
listener.stop();
}
}
@Test
public void testConnect() throws Exception {
when(mockServiceLocator.getService()).thenReturn(service);
when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
ConnectionResponseMessage mockMessage = new ConnectionResponseMessage();
mockMessage.setConnectionResponse(new ConnectionResponse(nodeIdentifier,
new StandardDataFlow("flow".getBytes("UTF-8"), new byte[0]), null, null, UUID.randomUUID().toString()));
when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(mockMessage);
ConnectionRequestMessage request = new ConnectionRequestMessage();
request.setConnectionRequest(new ConnectionRequest(nodeIdentifier));
ConnectionResponseMessage response = sender.requestConnection(request);
assertNotNull(response);
}
@Test(expected = UnknownServiceAddressException.class)
public void testConnectNoClusterManagerAddress() throws Exception {
when(mockServiceLocator.getService()).thenReturn(null);
when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(new ConnectionResponseMessage());
ConnectionRequestMessage request = new ConnectionRequestMessage();
request.setConnectionRequest(new ConnectionRequest(nodeIdentifier));
sender.requestConnection(request);
fail("failed to throw exception");
}
@Test(expected = ProtocolException.class)
public void testConnectBadResponse() throws Exception {
when(mockServiceLocator.getService()).thenReturn(service);
when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(new PingMessage());
ConnectionRequestMessage request = new ConnectionRequestMessage();
request.setConnectionRequest(new ConnectionRequest(nodeIdentifier));
sender.requestConnection(request);
fail("failed to throw exception");
}
@Test(expected = ProtocolException.class)
public void testConnectDelayedResponse() throws Exception {
final int time = 250;
sender.getSocketConfiguration().setSocketTimeout(time);
when(mockServiceLocator.getService()).thenReturn(service);
when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
when(mockHandler.handle(any(ProtocolMessage.class))).thenAnswer(new Answer<ConnectionResponseMessage>() {
@Override
public ConnectionResponseMessage answer(InvocationOnMock invocation) throws Throwable {
Thread.sleep(time * 3);
return new ConnectionResponseMessage();
}
});
ConnectionRequestMessage request = new ConnectionRequestMessage();
request.setConnectionRequest(new ConnectionRequest(nodeIdentifier));
sender.requestConnection(request);
fail("failed to throw exception");
}
}

View File

@ -0,0 +1,68 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.protocol.jaxb.message;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.util.Collections;
import java.util.List;
import javax.xml.bind.JAXBException;
import org.apache.nifi.cluster.coordination.node.DisconnectionCode;
import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus;
import org.apache.nifi.cluster.protocol.ComponentRevision;
import org.apache.nifi.cluster.protocol.ConnectionResponse;
import org.apache.nifi.cluster.protocol.DataFlow;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.cluster.protocol.StandardDataFlow;
import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage;
import org.apache.nifi.web.Revision;
import org.junit.Test;
public class TestJaxbProtocolUtils {
@Test
public void testRoundTripConnectionResponse() throws JAXBException {
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
final ConnectionResponseMessage msg = new ConnectionResponseMessage();
final NodeIdentifier nodeId = new NodeIdentifier("id", "localhost", 8000, "localhost", 8001, "localhost", 8002, true);
final DataFlow dataFlow = new StandardDataFlow(new byte[0], new byte[0]);
final List<NodeConnectionStatus> nodeStatuses = Collections.singletonList(new NodeConnectionStatus(nodeId, DisconnectionCode.NOT_YET_CONNECTED));
final List<ComponentRevision> componentRevisions = Collections.singletonList(ComponentRevision.fromRevision(new Revision(8L, "client-1", "component-1")));
msg.setConnectionResponse(new ConnectionResponse(nodeId, dataFlow, 80, false, "instance-1", nodeStatuses, componentRevisions));
JaxbProtocolUtils.JAXB_CONTEXT.createMarshaller().marshal(msg, baos);
final Object unmarshalled = JaxbProtocolUtils.JAXB_CONTEXT.createUnmarshaller().unmarshal(new ByteArrayInputStream(baos.toByteArray()));
assertTrue(unmarshalled instanceof ConnectionResponseMessage);
final ConnectionResponseMessage unmarshalledMsg = (ConnectionResponseMessage) unmarshalled;
final List<ComponentRevision> revisions = msg.getConnectionResponse().getComponentRevisions();
assertEquals(1, revisions.size());
assertEquals(8L, revisions.get(0).getVersion().longValue());
assertEquals("client-1", revisions.get(0).getClientId());
assertEquals("component-1", revisions.get(0).getComponentId());
assertEquals(revisions, unmarshalledMsg.getConnectionResponse().getComponentRevisions());
}
}

View File

@ -1,151 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.client;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.net.DatagramPacket;
import java.net.InetSocketAddress;
import java.net.MulticastSocket;
import java.text.SimpleDateFormat;
import java.util.Date;
import java.util.Locale;
import java.util.Timer;
import java.util.TimerTask;
import org.apache.nifi.cluster.protocol.ProtocolContext;
import org.apache.nifi.cluster.protocol.ProtocolException;
import org.apache.nifi.cluster.protocol.ProtocolHandler;
import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller;
import org.apache.nifi.cluster.protocol.impl.MulticastProtocolListener;
import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext;
import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils;
import org.apache.nifi.cluster.protocol.message.PingMessage;
import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
import org.apache.nifi.io.socket.multicast.MulticastConfiguration;
import org.apache.nifi.io.socket.multicast.MulticastUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Simple multicast test client that sends ping messages to a group address.
*/
public class MulticastTestClient {
private static final Logger logger = LoggerFactory.getLogger(MulticastTestClient.class);
private static final int PING_DELAY_SECONDS = 3;
public static void main(final String... args) throws IOException {
String group = System.getProperty("group", "225.0.0.0");
if (group == null) {
System.out.println("Host system property 'group' was not given.");
return;
}
group = group.trim();
if (group.length() == 0) {
System.out.println("Host system property 'group' must be non-empty.");
return;
}
final String portStr = System.getProperty("port", "2222");
final int port;
try {
port = Integer.parseInt(portStr);
} catch (final NumberFormatException nfe) {
System.out.println("Port system property 'port' was not a valid port.");
return;
}
logger.info(String.format("Pinging every %s seconds using multicast address: %s:%s.", PING_DELAY_SECONDS, group, port));
logger.info("Override defaults by using system properties '-Dgroup=<Class D IP>' and '-Dport=<unused port>'.");
logger.info("The test client may be stopped by entering a newline at the command line.");
final InetSocketAddress addr = new InetSocketAddress(group, port);
final ProtocolContext<ProtocolMessage> protocolContext = new JaxbProtocolContext<ProtocolMessage>(JaxbProtocolUtils.JAXB_CONTEXT);
final MulticastConfiguration multicastConfig = new MulticastConfiguration();
multicastConfig.setReuseAddress(true);
// setup listener
final MulticastProtocolListener listener = new MulticastProtocolListener(1, addr, multicastConfig, protocolContext);
listener.addHandler(new ProtocolHandler() {
@Override
public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException {
final PingMessage pingMsg = (PingMessage) msg;
final SimpleDateFormat sdf = new SimpleDateFormat("HH:mm:ss", Locale.US);
logger.info("Pinged at: " + sdf.format(pingMsg.getDate()));
return null;
}
@Override
public boolean canHandle(ProtocolMessage msg) {
return true;
}
});
// setup socket
final MulticastSocket multicastSocket = MulticastUtils.createMulticastSocket(multicastConfig);
// setup broadcaster
final Timer broadcaster = new Timer("Multicast Test Client", /**
* is daemon *
*/
true);
try {
// start listening
listener.start();
// start broadcasting
broadcaster.schedule(new TimerTask() {
@Override
public void run() {
try {
final PingMessage msg = new PingMessage();
msg.setDate(new Date());
// marshal message to output stream
final ProtocolMessageMarshaller<ProtocolMessage> marshaller = protocolContext.createMarshaller();
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
marshaller.marshal(msg, baos);
final byte[] packetBytes = baos.toByteArray();
// send message
final DatagramPacket packet = new DatagramPacket(packetBytes, packetBytes.length, addr);
multicastSocket.send(packet);
} catch (final Exception ex) {
logger.warn("Failed to send message due to: " + ex, ex);
}
}
}, 0, PING_DELAY_SECONDS * 1000);
// block until any input is received
System.in.read();
} finally {
broadcaster.cancel();
if (listener.isRunning()) {
listener.stop();
}
}
}
}

View File

@ -17,11 +17,6 @@
package org.apache.nifi.cluster.coordination.heartbeat;
import java.util.Map;
import java.util.Properties;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import org.apache.nifi.cluster.coordination.ClusterCoordinator;
import org.apache.nifi.cluster.coordination.node.DisconnectionCode;
import org.apache.nifi.cluster.coordination.node.NodeConnectionState;
@ -35,6 +30,11 @@ import org.apache.nifi.util.StopWatch;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Map;
import java.util.Properties;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
public abstract class AbstractHeartbeatMonitor implements HeartbeatMonitor {
private final int heartbeatIntervalMillis;
@ -55,9 +55,19 @@ public abstract class AbstractHeartbeatMonitor implements HeartbeatMonitor {
}
@Override
public final void start() {
public synchronized final void start() {
if (!stopped) {
throw new IllegalStateException("Heartbeat Monitor cannot be started because it is already started");
}
stopped = false;
onStart();
logger.info("Heartbeat Monitor started");
try {
onStart();
} catch (final Exception e) {
logger.error("Failed to start Heartbeat Monitor", e);
}
this.future = flowEngine.scheduleWithFixedDelay(new Runnable() {
@Override
@ -73,8 +83,13 @@ public abstract class AbstractHeartbeatMonitor implements HeartbeatMonitor {
}
@Override
public final void stop() {
public synchronized final void stop() {
if (stopped) {
return;
}
this.stopped = true;
logger.info("Heartbeat Monitor stopped");
try {
if (future != null) {
@ -112,7 +127,7 @@ public abstract class AbstractHeartbeatMonitor implements HeartbeatMonitor {
final Map<NodeIdentifier, NodeHeartbeat> latestHeartbeats = getLatestHeartbeats();
if (latestHeartbeats == null || latestHeartbeats.isEmpty()) {
// failed to fetch heartbeats; don't change anything.
clusterCoordinator.reportEvent(null, Severity.WARNING, "Failed to retrieve any new heartbeat information for nodes. "
clusterCoordinator.reportEvent(null, Severity.INFO, "Failed to retrieve any new heartbeat information for nodes. "
+ "Will not make any decisions based on heartbeats.");
return;
}
@ -133,8 +148,8 @@ public abstract class AbstractHeartbeatMonitor implements HeartbeatMonitor {
logger.info("Finished processing {} heartbeats in {}", latestHeartbeats.size(), procStopWatch.getDuration());
// Disconnect any node that hasn't sent a heartbeat in a long time (8 times the heartbeat interval)
final long maxMillis = heartbeatIntervalMillis * 1000L * 8;
final long threshold = latestHeartbeatTime - maxMillis;
final long maxMillis = heartbeatIntervalMillis * 8;
final long threshold = System.currentTimeMillis() - maxMillis;
for (final NodeHeartbeat heartbeat : latestHeartbeats.values()) {
if (heartbeat.getTimestamp() < threshold) {
clusterCoordinator.requestNodeDisconnect(heartbeat.getNodeIdentifier(), DisconnectionCode.LACK_OF_HEARTBEAT,
@ -165,52 +180,21 @@ public abstract class AbstractHeartbeatMonitor implements HeartbeatMonitor {
final NodeConnectionStatus connectionStatus = clusterCoordinator.getConnectionStatus(nodeId);
if (connectionStatus == null) {
final NodeConnectionState hbConnectionState = heartbeat.getConnectionStatus().getState();
if (hbConnectionState == NodeConnectionState.DISCONNECTED || hbConnectionState == NodeConnectionState.DISCONNECTING) {
// Node is not part of the cluster. Remove heartbeat and move on.
removeHeartbeat(nodeId);
return;
}
// Unknown node. Issue reconnect request
clusterCoordinator.reportEvent(nodeId, Severity.INFO, "Received heartbeat from unknown node. Removing heartbeat and requesting that node connect to cluster.");
removeHeartbeat(nodeId);
clusterCoordinator.requestNodeConnect(nodeId);
clusterCoordinator.requestNodeConnect(nodeId, null);
return;
}
final DisconnectionCode reportedDisconnectCode = heartbeat.getConnectionStatus().getDisconnectCode();
if (reportedDisconnectCode != null) {
// Check if the node is notifying us that it wants to disconnect from the cluster
final boolean requestingDisconnect;
switch (reportedDisconnectCode) {
case MISMATCHED_FLOWS:
case NODE_SHUTDOWN:
case STARTUP_FAILURE:
final NodeConnectionState expectedState = connectionStatus.getState();
requestingDisconnect = expectedState == NodeConnectionState.CONNECTED || expectedState == NodeConnectionState.CONNECTING;
break;
default:
requestingDisconnect = false;
break;
}
if (requestingDisconnect) {
clusterCoordinator.disconnectionRequestedByNode(nodeId, heartbeat.getConnectionStatus().getDisconnectCode(),
heartbeat.getConnectionStatus().getDisconnectReason());
removeHeartbeat(nodeId);
return;
}
}
final NodeConnectionState connectionState = connectionStatus.getState();
if (heartbeat.getConnectionStatus().getState() != NodeConnectionState.CONNECTED && connectionState == NodeConnectionState.CONNECTED) {
// Cluster Coordinator believes that node is connected, but node does not believe so.
clusterCoordinator.reportEvent(nodeId, Severity.WARNING, "Received heartbeat from node that thinks it is not yet part of the cluster,"
+ "though the Cluster Coordinator thought it was (node claimed state was " + heartbeat.getConnectionStatus().getState()
+ "). Marking as Disconnected and requesting that Node reconnect to cluster");
clusterCoordinator.requestNodeConnect(nodeId);
clusterCoordinator.requestNodeConnect(nodeId, null);
return;
}
@ -220,14 +204,14 @@ public abstract class AbstractHeartbeatMonitor implements HeartbeatMonitor {
// we cannot manually reconnect it.
final DisconnectionCode disconnectionCode = connectionStatus.getDisconnectCode();
if (disconnectionCode == DisconnectionCode.LACK_OF_HEARTBEAT) {
// record event
// Determine whether or not the node should be allowed to be in the cluster still, depending on its reason for disconnection.
if (disconnectionCode == DisconnectionCode.LACK_OF_HEARTBEAT || disconnectionCode == DisconnectionCode.UNABLE_TO_COMMUNICATE) {
clusterCoordinator.reportEvent(nodeId, Severity.INFO, "Received heartbeat from node previously "
+ "disconnected due to lack of heartbeat. Issuing reconnection request.");
+ "disconnected due to " + disconnectionCode + ". Issuing reconnection request.");
clusterCoordinator.requestNodeConnect(nodeId);
clusterCoordinator.requestNodeConnect(nodeId, null);
} else {
// disconnected nodes should not heartbeat, so we need to issue a disconnection request
// disconnected nodes should not heartbeat, so we need to issue a disconnection request.
logger.info("Ignoring received heartbeat from disconnected node " + nodeId + ". Issuing disconnection request.");
clusterCoordinator.requestNodeDisconnect(nodeId, connectionStatus.getDisconnectCode(), connectionStatus.getDisconnectReason());
removeHeartbeat(nodeId);
@ -256,9 +240,7 @@ public abstract class AbstractHeartbeatMonitor implements HeartbeatMonitor {
clusterCoordinator.reportEvent(nodeId, Severity.INFO, "Received first heartbeat from connecting node. Node connected.");
}
if (heartbeat.isPrimary()) {
clusterCoordinator.setPrimaryNode(nodeId);
}
clusterCoordinator.updateNodeRoles(nodeId, heartbeat.getRoles());
}

View File

@ -43,6 +43,7 @@ import org.apache.nifi.cluster.protocol.Heartbeat;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.cluster.protocol.ProtocolException;
import org.apache.nifi.cluster.protocol.ProtocolHandler;
import org.apache.nifi.cluster.protocol.ProtocolListener;
import org.apache.nifi.cluster.protocol.message.HeartbeatMessage;
import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType;
@ -55,7 +56,8 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Uses Apache Curator to monitor heartbeats from nodes
* Uses Apache ZooKeeper to advertise the address to send heartbeats to, and then relies on the NiFi Cluster
* Protocol to receive heartbeat messages from nodes in the cluster.
*/
public class ClusterProtocolHeartbeatMonitor extends AbstractHeartbeatMonitor implements HeartbeatMonitor, ProtocolHandler {
protected static final Logger logger = LoggerFactory.getLogger(ClusterProtocolHeartbeatMonitor.class);
@ -64,7 +66,6 @@ public class ClusterProtocolHeartbeatMonitor extends AbstractHeartbeatMonitor im
private final ZooKeeperClientConfig zkClientConfig;
private final String clusterNodesPath;
private volatile CuratorFramework curatorClient;
private volatile Map<String, NodeIdentifier> clusterNodeIds = new HashMap<>();
private final String heartbeatAddress;
@ -82,32 +83,33 @@ public class ClusterProtocolHeartbeatMonitor extends AbstractHeartbeatMonitor im
}
public ClusterProtocolHeartbeatMonitor(final ClusterCoordinator clusterCoordinator, final Properties properties) {
public ClusterProtocolHeartbeatMonitor(final ClusterCoordinator clusterCoordinator, final ProtocolListener protocolListener, final Properties properties) {
super(clusterCoordinator, properties);
protocolListener.addHandler(this);
this.zkClientConfig = ZooKeeperClientConfig.createConfig(properties);
this.clusterNodesPath = zkClientConfig.resolvePath("cluster/nodes");
String hostname = properties.getProperty(NiFiProperties.CLUSTER_MANAGER_ADDRESS);
String hostname = properties.getProperty(NiFiProperties.CLUSTER_NODE_ADDRESS);
if (hostname == null) {
try {
hostname = InetAddress.getLocalHost().getHostName();
} catch (UnknownHostException e) {
throw new RuntimeException("Unable to determine local hostname and the '" + NiFiProperties.CLUSTER_MANAGER_ADDRESS + "' property is not set");
throw new RuntimeException("Unable to determine local hostname and the '" + NiFiProperties.CLUSTER_NODE_ADDRESS + "' property is not set");
}
}
final String port = properties.getProperty(NiFiProperties.CLUSTER_MANAGER_PROTOCOL_PORT);
final String port = properties.getProperty(NiFiProperties.CLUSTER_NODE_PROTOCOL_PORT);
if (port == null) {
throw new RuntimeException("Unable to determine which port Cluster Manager Protocol is listening on because the '"
+ NiFiProperties.CLUSTER_MANAGER_PROTOCOL_PORT + "' property is not set");
throw new RuntimeException("Unable to determine which port Cluster Coordinator Protocol is listening on because the '"
+ NiFiProperties.CLUSTER_NODE_PROTOCOL_PORT + "' property is not set");
}
try {
Integer.parseInt(port);
} catch (final NumberFormatException nfe) {
throw new RuntimeException("Unable to determine which port Cluster Manager Protocol is listening on because the '"
+ NiFiProperties.CLUSTER_MANAGER_PROTOCOL_PORT + "' property is set to '" + port + "', which is not a valid port number.");
throw new RuntimeException("Unable to determine which port Cluster Coordinator Protocol is listening on because the '"
+ NiFiProperties.CLUSTER_NODE_PROTOCOL_PORT + "' property is set to '" + port + "', which is not a valid port number.");
}
heartbeatAddress = hostname + ":" + port;
@ -116,10 +118,22 @@ public class ClusterProtocolHeartbeatMonitor extends AbstractHeartbeatMonitor im
@Override
public void onStart() {
final RetryPolicy retryPolicy = new RetryForever(5000);
curatorClient = CuratorFrameworkFactory.newClient(zkClientConfig.getConnectString(),
final CuratorFramework curatorClient = CuratorFrameworkFactory.newClient(zkClientConfig.getConnectString(),
zkClientConfig.getSessionTimeoutMillis(), zkClientConfig.getConnectionTimeoutMillis(), retryPolicy);
curatorClient.start();
// We don't know what the heartbeats look like for the nodes, since we were just elected to monitoring
// them. However, the map may be filled with old heartbeats. So we clear the heartbeats and populate the
// map with new heartbeats set to the current time and using the currently known status. We do this so
// that if we go the required amount of time without receiving a heartbeat, we do know to mark the node
// as disconnected.
heartbeatMessages.clear();
for (final NodeIdentifier nodeId : clusterCoordinator.getNodeIdentifiers()) {
final NodeHeartbeat heartbeat = new StandardNodeHeartbeat(nodeId, System.currentTimeMillis(),
clusterCoordinator.getConnectionStatus(nodeId), Collections.emptySet(), 0, 0L, 0, System.currentTimeMillis());
heartbeatMessages.put(nodeId, heartbeat);
}
final Thread publishAddress = new Thread(new Runnable() {
@Override
public void run() {
@ -128,6 +142,8 @@ public class ClusterProtocolHeartbeatMonitor extends AbstractHeartbeatMonitor im
try {
try {
curatorClient.setData().forPath(path, heartbeatAddress.getBytes(StandardCharsets.UTF_8));
curatorClient.close();
logger.info("Successfully published Cluster Heartbeat Monitor Address of {} to ZooKeeper", heartbeatAddress);
return;
} catch (final NoNodeException nne) {
// ensure that parents are created, using a wide-open ACL because the parents contain no data
@ -162,16 +178,8 @@ public class ClusterProtocolHeartbeatMonitor extends AbstractHeartbeatMonitor im
publishAddress.start();
}
private CuratorFramework getClient() {
return curatorClient;
}
@Override
public void onStop() {
final CuratorFramework client = getClient();
if (client != null) {
client.close();
}
}
@Override
@ -201,7 +209,7 @@ public class ClusterProtocolHeartbeatMonitor extends AbstractHeartbeatMonitor im
final NodeIdentifier nodeId = heartbeat.getNodeIdentifier();
final NodeConnectionStatus connectionStatus = heartbeat.getConnectionStatus();
final boolean primary = heartbeat.isPrimary();
final Set<String> roles = heartbeat.getRoles();
final byte[] payloadBytes = heartbeat.getPayload();
final HeartbeatPayload payload = HeartbeatPayload.unmarshal(payloadBytes);
final int activeThreadCount = payload.getActiveThreadCount();
@ -210,7 +218,7 @@ public class ClusterProtocolHeartbeatMonitor extends AbstractHeartbeatMonitor im
final long systemStartTime = payload.getSystemStartTime();
final NodeHeartbeat nodeHeartbeat = new StandardNodeHeartbeat(nodeId, System.currentTimeMillis(),
connectionStatus, primary, flowFileCount, flowFileBytes, activeThreadCount, systemStartTime);
connectionStatus, roles, flowFileCount, flowFileBytes, activeThreadCount, systemStartTime);
heartbeatMessages.put(heartbeat.getNodeIdentifier(), nodeHeartbeat);
logger.debug("Received new heartbeat from {}", nodeId);

View File

@ -1,376 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.coordination.heartbeat;
import java.io.ByteArrayInputStream;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import javax.xml.bind.JAXBContext;
import javax.xml.bind.Unmarshaller;
import org.apache.curator.RetryPolicy;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.CuratorFrameworkFactory;
import org.apache.curator.retry.RetryForever;
import org.apache.nifi.cluster.coordination.ClusterCoordinator;
import org.apache.nifi.cluster.coordination.node.DisconnectionCode;
import org.apache.nifi.cluster.coordination.node.NodeConnectionState;
import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.cluster.protocol.message.HeartbeatMessage;
import org.apache.nifi.controller.cluster.ZooKeeperClientConfig;
import org.apache.nifi.engine.FlowEngine;
import org.apache.nifi.reporting.Severity;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.util.StopWatch;
import org.apache.zookeeper.KeeperException.NoNodeException;
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Uses Apache Curator to monitor heartbeats from nodes
*/
public class CuratorHeartbeatMonitor implements HeartbeatMonitor {
private static final Logger logger = LoggerFactory.getLogger(CuratorHeartbeatMonitor.class);
private static final Unmarshaller unmarshaller;
private final ClusterCoordinator clusterCoordinator;
private final ZooKeeperClientConfig zkClientConfig;
private final String heartbeatPath;
private final int heartbeatIntervalMillis;
private volatile CuratorFramework curatorClient;
private volatile ScheduledFuture<?> future;
private volatile Map<NodeIdentifier, NodeHeartbeat> latestHeartbeatMessages;
private volatile long latestHeartbeatTime;
private final FlowEngine flowEngine = new FlowEngine(1, "Heartbeat Monitor", true);
static {
try {
final JAXBContext jaxbContext = JAXBContext.newInstance(HeartbeatMessage.class);
unmarshaller = jaxbContext.createUnmarshaller();
} catch (final Exception e) {
throw new RuntimeException("Failed to create an Unmarshaller for unmarshalling Heartbeat Messages", e);
}
}
public CuratorHeartbeatMonitor(final ClusterCoordinator clusterCoordinator, final Properties properties) {
this.clusterCoordinator = clusterCoordinator;
this.zkClientConfig = ZooKeeperClientConfig.createConfig(properties);
this.heartbeatPath = zkClientConfig.resolvePath("cluster/heartbeats");
final String heartbeatInterval = properties.getProperty(NiFiProperties.CLUSTER_PROTOCOL_HEARTBEAT_INTERVAL,
NiFiProperties.DEFAULT_CLUSTER_PROTOCOL_HEARTBEAT_INTERVAL);
this.heartbeatIntervalMillis = (int) FormatUtils.getTimeDuration(heartbeatInterval, TimeUnit.MILLISECONDS);
}
@Override
public void start() {
final RetryPolicy retryPolicy = new RetryForever(5000);
curatorClient = CuratorFrameworkFactory.newClient(zkClientConfig.getConnectString(),
zkClientConfig.getSessionTimeoutMillis(), zkClientConfig.getConnectionTimeoutMillis(), retryPolicy);
curatorClient.start();
this.future = flowEngine.scheduleWithFixedDelay(new Runnable() {
@Override
public void run() {
try {
monitorHeartbeats();
} catch (final Exception e) {
clusterCoordinator.reportEvent(null, Severity.ERROR, "Failed to process heartbeats from nodes due to " + e.toString());
logger.error("Failed to process heartbeats", e);
}
}
}, heartbeatIntervalMillis, heartbeatIntervalMillis, TimeUnit.MILLISECONDS);
}
private CuratorFramework getClient() {
return curatorClient;
}
@Override
public void stop() {
final CuratorFramework client = getClient();
if (client != null) {
client.close();
}
if (future != null) {
future.cancel(true);
}
}
@Override
public NodeHeartbeat getLatestHeartbeat(final NodeIdentifier nodeId) {
return latestHeartbeatMessages.get(nodeId);
}
/**
* Fetches all of the latest heartbeats from ZooKeeper
* and updates the Cluster Coordinator as appropriate,
* based on the heartbeats received.
*
* Visible for testing.
*/
synchronized void monitorHeartbeats() {
final StopWatch fetchStopWatch = new StopWatch(true);
final Map<NodeIdentifier, NodeHeartbeat> latestHeartbeats = fetchHeartbeats();
if (latestHeartbeats == null || latestHeartbeats.isEmpty()) {
// failed to fetch heartbeats from ZooKeeper; don't change anything.
clusterCoordinator.reportEvent(null, Severity.WARNING, "Failed to retrieve any new heartbeat information for nodes from ZooKeeper. "
+ "Will not make any decisions based on heartbeats.");
return;
}
this.latestHeartbeatMessages = new HashMap<>(latestHeartbeats);
fetchStopWatch.stop();
final StopWatch procStopWatch = new StopWatch(true);
for (final NodeHeartbeat heartbeat : latestHeartbeats.values()) {
try {
processHeartbeat(heartbeat);
} catch (final Exception e) {
clusterCoordinator.reportEvent(null, Severity.ERROR,
"Received heartbeat from " + heartbeat.getNodeIdentifier() + " but failed to process heartbeat due to " + e);
logger.error("Failed to process heartbeat from {} due to {}", heartbeat.getNodeIdentifier(), e.toString());
logger.error("", e);
}
}
procStopWatch.stop();
logger.info("Finished processing {} heartbeats in {} (fetch took an additional {})",
latestHeartbeats.size(), procStopWatch.getDuration(), fetchStopWatch.getDuration());
// Disconnect any node that hasn't sent a heartbeat in a long time (8 times the heartbeat interval)
for (final NodeHeartbeat heartbeat : latestHeartbeats.values()) {
final long maxMillis = heartbeatIntervalMillis * 1000L * 8;
final long threshold = latestHeartbeatTime - maxMillis;
if (heartbeat.getTimestamp() < threshold) {
final int differenceSeconds = (int) TimeUnit.MILLISECONDS.toSeconds(latestHeartbeatTime - heartbeat.getTimestamp());
clusterCoordinator.requestNodeDisconnect(heartbeat.getNodeIdentifier(), DisconnectionCode.LACK_OF_HEARTBEAT,
"Node has not sent a heartbeat to ZooKeeper in " + differenceSeconds + " seconds");
try {
removeHeartbeat(heartbeat.getNodeIdentifier());
} catch (final Exception e) {
logger.warn("Failed to remove heartbeat for {} due to {}", heartbeat.getNodeIdentifier(), e.toString());
logger.warn("", e);
}
}
}
}
private void processHeartbeat(final NodeHeartbeat heartbeat) {
final NodeIdentifier nodeId = heartbeat.getNodeIdentifier();
// Do not process heartbeat if it's blocked by firewall.
if (clusterCoordinator.isBlockedByFirewall(nodeId.getSocketAddress())) {
clusterCoordinator.reportEvent(nodeId, Severity.WARNING, "Firewall blocked received heartbeat. Issuing disconnection request.");
// request node to disconnect
clusterCoordinator.requestNodeDisconnect(nodeId, DisconnectionCode.BLOCKED_BY_FIREWALL, "Blocked by Firewall");
removeHeartbeat(nodeId);
return;
}
final NodeConnectionStatus connectionStatus = clusterCoordinator.getConnectionStatus(nodeId);
if (connectionStatus == null) {
final NodeConnectionState hbConnectionState = heartbeat.getConnectionStatus().getState();
if (hbConnectionState == NodeConnectionState.DISCONNECTED || hbConnectionState == NodeConnectionState.DISCONNECTING) {
// Node is not part of the cluster. Remove heartbeat and move on.
removeHeartbeat(nodeId);
return;
}
// Unknown node. Issue reconnect request
clusterCoordinator.reportEvent(nodeId, Severity.INFO, "Received heartbeat from unknown node. Removing heartbeat and requesting that node connect to cluster.");
removeHeartbeat(nodeId);
clusterCoordinator.requestNodeConnect(nodeId);
return;
}
final DisconnectionCode reportedDisconnectCode = heartbeat.getConnectionStatus().getDisconnectCode();
if (reportedDisconnectCode != null) {
// Check if the node is notifying us that it wants to disconnect from the cluster
final boolean requestingDisconnect;
switch (reportedDisconnectCode) {
case MISMATCHED_FLOWS:
case NODE_SHUTDOWN:
case STARTUP_FAILURE:
final NodeConnectionState expectedState = connectionStatus.getState();
requestingDisconnect = expectedState == NodeConnectionState.CONNECTED || expectedState == NodeConnectionState.CONNECTING;
break;
default:
requestingDisconnect = false;
break;
}
if (requestingDisconnect) {
clusterCoordinator.disconnectionRequestedByNode(nodeId, heartbeat.getConnectionStatus().getDisconnectCode(),
heartbeat.getConnectionStatus().getDisconnectReason());
removeHeartbeat(nodeId);
return;
}
}
final NodeConnectionState connectionState = connectionStatus.getState();
if (heartbeat.getConnectionStatus().getState() != NodeConnectionState.CONNECTED && connectionState == NodeConnectionState.CONNECTED) {
// Cluster Coordinator believes that node is connected, but node does not believe so.
clusterCoordinator.reportEvent(nodeId, Severity.WARNING, "Received heartbeat from node that thinks it is not yet part of the cluster,"
+ "though the Cluster Coordinator thought it was (node claimed state was " + heartbeat.getConnectionStatus().getState()
+ "). Marking as Disconnected and requesting that Node reconnect to cluster");
clusterCoordinator.requestNodeConnect(nodeId);
return;
}
if (NodeConnectionState.DISCONNECTED == connectionState) {
// ignore heartbeats from nodes disconnected by means other than lack of heartbeat, unless it is
// the only node. We allow it if it is the only node because if we have a one-node cluster, then
// we cannot manually reconnect it.
final DisconnectionCode disconnectionCode = connectionStatus.getDisconnectCode();
if (disconnectionCode == DisconnectionCode.LACK_OF_HEARTBEAT) {
// record event
clusterCoordinator.reportEvent(nodeId, Severity.INFO, "Received heartbeat from node previously "
+ "disconnected due to lack of heartbeat. Issuing reconnection request.");
clusterCoordinator.requestNodeConnect(nodeId);
} else {
// disconnected nodes should not heartbeat, so we need to issue a disconnection request
logger.info("Ignoring received heartbeat from disconnected node " + nodeId + ". Issuing disconnection request.");
clusterCoordinator.requestNodeDisconnect(nodeId, connectionStatus.getDisconnectCode(), connectionStatus.getDisconnectReason());
removeHeartbeat(nodeId);
}
return;
}
if (NodeConnectionState.DISCONNECTING == connectionStatus.getState()) {
// ignore spurious heartbeat
removeHeartbeat(nodeId);
return;
}
// first heartbeat causes status change from connecting to connected
if (NodeConnectionState.CONNECTING == connectionState) {
final Long connectionRequestTime = connectionStatus.getConnectionRequestTime();
if (connectionRequestTime != null && heartbeat.getTimestamp() < connectionRequestTime) {
clusterCoordinator.reportEvent(nodeId, Severity.INFO, "Received heartbeat but ignoring because it was reported before the node was last asked to reconnect.");
removeHeartbeat(nodeId);
return;
}
// connection complete
clusterCoordinator.finishNodeConnection(nodeId);
clusterCoordinator.reportEvent(nodeId, Severity.INFO, "Received first heartbeat from connecting node. Node connected.");
}
if (heartbeat.isPrimary()) {
clusterCoordinator.setPrimaryNode(nodeId);
}
}
/**
* Fetches the latest heartbeats for each node from ZooKeeper.
* Visible for testing
*/
Map<NodeIdentifier, NodeHeartbeat> fetchHeartbeats() {
logger.debug("Fetching heartbeats from ZooKeeper");
final List<String> nodeIds;
try {
nodeIds = curatorClient.getChildren().forPath(heartbeatPath);
} catch (final NoNodeException nne) {
logger.info("Could not find any heartbeats in ZooKeeper because the ZNode " + heartbeatPath + " does not exist");
return null;
} catch (final Exception e) {
logger.error("Failed to obtain heartbeats from ZooKeeper due to {}", e);
logger.error("", e);
clusterCoordinator.reportEvent(null, Severity.WARNING, "Failed to obtain heartbeats from ZooKeeper due to " + e);
return null;
}
logger.debug("Found {} nodes that have emitted heartbeats to ZooKeeper", nodeIds.size());
final Map<NodeIdentifier, NodeHeartbeat> latestHeartbeats = new HashMap<>(nodeIds.size());
for (final String nodeId : nodeIds) {
final HeartbeatMessage heartbeatMsg;
final String nodeHeartbeatPath = heartbeatPath + "/" + nodeId;
final Stat stat = new Stat();
try {
final byte[] serializedHeartbeatMsg = getClient().getData().storingStatIn(stat).forPath(nodeHeartbeatPath);
heartbeatMsg = (HeartbeatMessage) unmarshaller.unmarshal(new ByteArrayInputStream(serializedHeartbeatMsg));
} catch (final Exception e) {
logger.error("Failed to obtain heartbeat from ZooKeeper for Node with ID {} due to {}", nodeId, e);
logger.error("", e);
clusterCoordinator.reportEvent(null, Severity.WARNING, "Failed to obtain heartbeat for Node with ID " + nodeId + " from ZooKeeper due to " + e);
continue;
}
// update timestamp to be the timestamp that ZooKeeper reports
final long lastModifiedTime = stat.getMtime();
if (lastModifiedTime > latestHeartbeatTime) {
latestHeartbeatTime = lastModifiedTime;
}
latestHeartbeats.put(heartbeatMsg.getHeartbeat().getNodeIdentifier(), StandardNodeHeartbeat.fromHeartbeatMessage(heartbeatMsg, lastModifiedTime));
logger.debug("Received heartbeat from Node {}", nodeId);
}
logger.debug("Fetched {} heartbeats from ZooKeeper", latestHeartbeats.size());
return latestHeartbeats;
}
@Override
public synchronized void removeHeartbeat(final NodeIdentifier nodeId) {
logger.debug("Deleting heartbeat for node {}", nodeId);
final String nodeHeartbeatPath = heartbeatPath + "/" + nodeId.getId();
latestHeartbeatMessages.remove(nodeId);
try {
getClient().delete().forPath(nodeHeartbeatPath);
logger.info("Removed heartbeat from ZooKeeper for Node {}", nodeId);
} catch (final NoNodeException e) {
// node did not exist. Just return.
logger.debug("Attempted to remove heartbeat for Node with ID {} but no ZNode existed at {}", nodeId, nodeHeartbeatPath);
return;
} catch (final Exception e) {
logger.warn("Failed to remove heartbeat from ZooKeeper for Node {} due to {}", nodeId, e);
logger.warn("", e);
clusterCoordinator.reportEvent(nodeId, Severity.WARNING, "Failed to remove node's heartbeat from ZooKeeper due to " + e);
}
}
}

View File

@ -17,6 +17,10 @@
package org.apache.nifi.cluster.coordination.heartbeat;
import java.util.Collections;
import java.util.HashSet;
import java.util.Set;
import org.apache.nifi.cluster.HeartbeatPayload;
import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus;
import org.apache.nifi.cluster.protocol.Heartbeat;
@ -28,18 +32,18 @@ public class StandardNodeHeartbeat implements NodeHeartbeat {
private final NodeIdentifier nodeId;
private final long timestamp;
private final NodeConnectionStatus connectionStatus;
private final boolean primary;
private final Set<String> roles;
private final int flowFileCount;
private final long flowFileBytes;
private final int activeThreadCount;
private final long systemStartTime;
public StandardNodeHeartbeat(final NodeIdentifier nodeId, final long timestamp, final NodeConnectionStatus connectionStatus,
final boolean primary, final int flowFileCount, final long flowFileBytes, final int activeThreadCount, final long systemStartTime) {
final Set<String> roles, final int flowFileCount, final long flowFileBytes, final int activeThreadCount, final long systemStartTime) {
this.timestamp = timestamp;
this.nodeId = nodeId;
this.connectionStatus = connectionStatus;
this.primary = primary;
this.roles = roles == null ? Collections.emptySet() : Collections.unmodifiableSet(new HashSet<>(roles));
this.flowFileCount = flowFileCount;
this.flowFileBytes = flowFileBytes;
this.activeThreadCount = activeThreadCount;
@ -62,8 +66,8 @@ public class StandardNodeHeartbeat implements NodeHeartbeat {
}
@Override
public boolean isPrimary() {
return primary;
public Set<String> getRoles() {
return roles;
}
@Override
@ -92,7 +96,7 @@ public class StandardNodeHeartbeat implements NodeHeartbeat {
final HeartbeatPayload payload = HeartbeatPayload.unmarshal(heartbeat.getPayload());
return new StandardNodeHeartbeat(heartbeat.getNodeIdentifier(), timestamp, heartbeat.getConnectionStatus(),
heartbeat.isPrimary(), (int) payload.getTotalFlowFileCount(), payload.getTotalFlowFileBytes(),
heartbeat.getRoles(), (int) payload.getTotalFlowFileCount(), payload.getTotalFlowFileBytes(),
payload.getActiveThreadCount(), payload.getSystemStartTime());
}
}

View File

@ -35,6 +35,7 @@ import org.apache.nifi.cluster.coordination.http.endpoints.ControllerServicesEnd
import org.apache.nifi.cluster.coordination.http.endpoints.ControllerStatusEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.CountersEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.DropRequestEndpiontMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.FlowMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.FlowSnippetEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.GroupStatusEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.ListFlowFilesEndpointMerger;
@ -52,25 +53,15 @@ import org.apache.nifi.cluster.coordination.http.endpoints.ReportingTaskEndpoint
import org.apache.nifi.cluster.coordination.http.endpoints.ReportingTasksEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.StatusHistoryEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.SystemDiagnosticsEndpointMerger;
import org.apache.nifi.cluster.coordination.http.replication.RequestReplicator;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.manager.StatusMerger;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import org.apache.nifi.cluster.node.Node.Status;
import org.apache.nifi.reporting.Bulletin;
import org.apache.nifi.reporting.BulletinQuery;
import org.apache.nifi.reporting.ComponentType;
import org.apache.nifi.stream.io.NullOutputStream;
import org.apache.nifi.web.api.dto.status.ControllerStatusDTO;
import org.apache.nifi.web.api.entity.ControllerStatusEntity;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class StandardHttpResponseMerger implements HttpResponseMerger {
private Logger logger = LoggerFactory.getLogger(StandardHttpResponseMerger.class);
private static final int NODE_CONTINUE_STATUS_CODE = 150;
private final WebClusterManager clusterManager;
private static final List<EndpointResponseMerger> endpointMergers = new ArrayList<>();
static {
endpointMergers.add(new ControllerStatusEndpointMerger());
@ -99,14 +90,10 @@ public class StandardHttpResponseMerger implements HttpResponseMerger {
endpointMergers.add(new StatusHistoryEndpointMerger());
endpointMergers.add(new SystemDiagnosticsEndpointMerger());
endpointMergers.add(new CountersEndpointMerger());
endpointMergers.add(new FlowMerger());
}
public StandardHttpResponseMerger() {
this(null);
}
public StandardHttpResponseMerger(final WebClusterManager clusterManager) {
this.clusterManager = clusterManager;
}
@Override
@ -140,58 +127,9 @@ public class StandardHttpResponseMerger implements HttpResponseMerger {
}
final NodeResponse response = merger.merge(uri, httpMethod, successResponses, problematicResponses, clientResponse);
if (clusterManager != null) {
mergeNCMBulletins(response, uri, httpMethod);
}
return response;
}
/**
* This method merges bulletins from the NCM. Eventually, the NCM will go away entirely, and
* at that point, we will completely remove this and the WebClusterManager as a member variable.
* However, until then, the bulletins from the NCM are important to include, since there is no other
* node that can include them.
*
* @param clientResponse the Node Response that will be returned to the client
* @param uri the URI
* @param method the HTTP Method
*
* @deprecated this method exists only until we can remove the Cluster Manager from the picture all together. It will then be removed.
*/
@Deprecated
private void mergeNCMBulletins(final NodeResponse clientResponse, final URI uri, final String method) {
// determine if we have at least one response
final boolean hasClientResponse = clientResponse != null;
final boolean hasSuccessfulClientResponse = hasClientResponse && clientResponse.is2xx();
if (hasSuccessfulClientResponse && clusterManager.isControllerStatusEndpoint(uri, method)) {
// for now, we need to merge the NCM's bulletins too.
final ControllerStatusEntity responseEntity = (ControllerStatusEntity) clientResponse.getUpdatedEntity();
final ControllerStatusDTO mergedStatus = responseEntity.getControllerStatus();
final int totalNodeCount = clusterManager.getNodeIds().size();
final int connectedNodeCount = clusterManager.getNodeIds(Status.CONNECTED).size();
final List<Bulletin> ncmControllerBulletins = clusterManager.getBulletinRepository().findBulletinsForController();
mergedStatus.setBulletins(clusterManager.mergeNCMBulletins(mergedStatus.getBulletins(), ncmControllerBulletins));
// get the controller service bulletins
final BulletinQuery controllerServiceQuery = new BulletinQuery.Builder().sourceType(ComponentType.CONTROLLER_SERVICE).build();
final List<Bulletin> ncmServiceBulletins = clusterManager.getBulletinRepository().findBulletins(controllerServiceQuery);
mergedStatus.setControllerServiceBulletins(clusterManager.mergeNCMBulletins(mergedStatus.getControllerServiceBulletins(), ncmServiceBulletins));
// get the reporting task bulletins
final BulletinQuery reportingTaskQuery = new BulletinQuery.Builder().sourceType(ComponentType.REPORTING_TASK).build();
final List<Bulletin> ncmReportingTaskBulletins = clusterManager.getBulletinRepository().findBulletins(reportingTaskQuery);
mergedStatus.setReportingTaskBulletins(clusterManager.mergeNCMBulletins(mergedStatus.getReportingTaskBulletins(), ncmReportingTaskBulletins));
mergedStatus.setConnectedNodeCount(connectedNodeCount);
mergedStatus.setTotalNodeCount(totalNodeCount);
StatusMerger.updatePrettyPrintedFields(mergedStatus);
}
}
@Override
public Set<NodeResponse> getProblematicNodeResponses(final Set<NodeResponse> allResponses) {
@ -225,7 +163,7 @@ public class StandardHttpResponseMerger implements HttpResponseMerger {
responses.stream()
.parallel() // parallelize the draining of the responses, since we have multiple streams to consume
.filter(response -> response != exclude) // don't include the explicitly excluded node
.filter(response -> response.getStatus() != NODE_CONTINUE_STATUS_CODE) // don't include any 150-NodeContinue responses because they contain no content
.filter(response -> response.getStatus() != RequestReplicator.NODE_CONTINUE_STATUS_CODE) // don't include any 150-NodeContinue responses because they contain no content
.forEach(response -> drainResponse(response)); // drain all node responses that didn't get filtered out
}

View File

@ -24,11 +24,16 @@ import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.web.api.entity.Entity;
public abstract class AbstractNodeStatusEndpoint<EntityType extends Entity, DtoType> extends AbstractSingleEntityEndpoint<EntityType, DtoType> {
public abstract class AbstractNodeStatusEndpoint<EntityType extends Entity, DtoType> extends AbstractSingleDTOEndpoint<EntityType, DtoType> {
@Override
protected final void mergeResponses(DtoType clientDto, Map<NodeIdentifier, DtoType> dtoMap, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses) {
final NodeIdentifier selectedNodeId = dtoMap.entrySet().stream().filter(e -> e.getValue() == clientDto).map(e -> e.getKey()).findFirst().orElse(null);
final NodeIdentifier selectedNodeId = dtoMap.entrySet().stream()
.filter(e -> e.getValue() == clientDto)
.map(e -> e.getKey())
.findFirst()
.orElse(null);
if (selectedNodeId == null) {
throw new IllegalArgumentException("Attempted to merge Status request but could not find the appropriate Node Identifier");
}

View File

@ -0,0 +1,115 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.coordination.http.endpoints;
import java.net.URI;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.web.api.entity.Entity;
public abstract class AbstractSingleDTOEndpoint<EntityType extends Entity, DtoType> implements EndpointResponseMerger {
@Override
public final NodeResponse merge(final URI uri, final String method, final Set<NodeResponse> successfulResponses, final Set<NodeResponse> problematicResponses, final NodeResponse clientResponse) {
if (!canHandle(uri, method)) {
throw new IllegalArgumentException("Cannot use Endpoint Mapper of type " + getClass().getSimpleName() + " to map responses for URI " + uri + ", HTTP Method " + method);
}
final EntityType responseEntity = clientResponse.getClientResponse().getEntity(getEntityClass());
final DtoType dto = getDto(responseEntity);
final Map<NodeIdentifier, DtoType> dtoMap = new HashMap<>();
for (final NodeResponse nodeResponse : successfulResponses) {
final EntityType nodeResponseEntity = nodeResponse == clientResponse ? responseEntity : nodeResponse.getClientResponse().getEntity(getEntityClass());
final DtoType nodeDto = getDto(nodeResponseEntity);
dtoMap.put(nodeResponse.getNodeId(), nodeDto);
}
mergeResponses(dto, dtoMap, successfulResponses, problematicResponses);
return new NodeResponse(clientResponse, responseEntity);
}
/**
* Normalizes the validation errors by prepending the corresponding nodes when the error does not exist across all nodes.
*
* @param validationErrorMap map
* @param totalNodes total
* @return normalized errors
*/
protected Set<String> normalizedMergedValidationErrors(final Map<String, Set<NodeIdentifier>> validationErrorMap, int totalNodes) {
final Set<String> normalizedValidationErrors = new HashSet<>();
for (final Map.Entry<String, Set<NodeIdentifier>> validationEntry : validationErrorMap.entrySet()) {
final String msg = validationEntry.getKey();
final Set<NodeIdentifier> nodeIds = validationEntry.getValue();
if (nodeIds.size() == totalNodes) {
normalizedValidationErrors.add(msg);
} else {
nodeIds.forEach(id -> normalizedValidationErrors.add(id.getApiAddress() + ":" + id.getApiPort() + " -- " + msg));
}
}
return normalizedValidationErrors;
}
/**
* Merges the validation errors into the specified map, recording the corresponding node identifier.
*
* @param validationErrorMap map
* @param nodeId id
* @param nodeValidationErrors errors
*/
protected void mergeValidationErrors(final Map<String, Set<NodeIdentifier>> validationErrorMap, final NodeIdentifier nodeId, final Collection<String> nodeValidationErrors) {
if (nodeValidationErrors != null) {
nodeValidationErrors.stream().forEach(
err -> validationErrorMap.computeIfAbsent(err, k -> new HashSet<NodeIdentifier>())
.add(nodeId));
}
}
/**
* @return the class that represents the type of Entity that is expected by this response mapper
*/
protected abstract Class<EntityType> getEntityClass();
/**
* Extracts the DTO from the given entity
*
* @param entity the entity to extract the DTO from
* @return the DTO from the given entity
*/
protected abstract DtoType getDto(EntityType entity);
/**
* Merges the responses from all nodes in the given map into the single given DTO
*
* @param clientDto the DTO to merge responses into
* @param dtoMap the responses from all nodes
* @param successfulResponses the responses from nodes that completed the request successfully
* @param problematicResponses the responses from nodes that did not complete the request successfully
*/
protected abstract void mergeResponses(DtoType clientDto, Map<NodeIdentifier, DtoType> dtoMap, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses);
}

View File

@ -29,8 +29,7 @@ import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.web.api.entity.Entity;
public abstract class AbstractSingleEntityEndpoint<EntityType extends Entity, DtoType> implements EndpointResponseMerger {
public abstract class AbstractSingleEntityEndpoint<EntityType extends Entity> implements EndpointResponseMerger {
@Override
public final NodeResponse merge(final URI uri, final String method, final Set<NodeResponse> successfulResponses, final Set<NodeResponse> problematicResponses, final NodeResponse clientResponse) {
if (!canHandle(uri, method)) {
@ -38,19 +37,31 @@ public abstract class AbstractSingleEntityEndpoint<EntityType extends Entity, Dt
}
final EntityType responseEntity = clientResponse.getClientResponse().getEntity(getEntityClass());
final DtoType dto = getDto(responseEntity);
final Map<NodeIdentifier, DtoType> dtoMap = new HashMap<>();
final Map<NodeIdentifier, EntityType> entityMap = new HashMap<>();
for (final NodeResponse nodeResponse : successfulResponses) {
final EntityType nodeResponseEntity = nodeResponse == clientResponse ? responseEntity : nodeResponse.getClientResponse().getEntity(getEntityClass());
final DtoType nodeDto = getDto(nodeResponseEntity);
dtoMap.put(nodeResponse.getNodeId(), nodeDto);
entityMap.put(nodeResponse.getNodeId(), nodeResponseEntity);
}
mergeResponses(dto, dtoMap, successfulResponses, problematicResponses);
mergeResponses(responseEntity, entityMap, successfulResponses, problematicResponses);
return new NodeResponse(clientResponse, responseEntity);
}
/**
* Merges the validation errors into the specified map, recording the corresponding node identifier.
*
* @param validationErrorMap map
* @param nodeId id
* @param nodeValidationErrors errors
*/
protected void mergeValidationErrors(final Map<String, Set<NodeIdentifier>> validationErrorMap, final NodeIdentifier nodeId, final Collection<String> nodeValidationErrors) {
if (nodeValidationErrors != null) {
nodeValidationErrors.stream().forEach(
err -> validationErrorMap.computeIfAbsent(err, k -> new HashSet<NodeIdentifier>())
.add(nodeId));
}
}
/**
* Normalizes the validation errors by prepending the corresponding nodes when the error does not exist across all nodes.
@ -74,42 +85,19 @@ public abstract class AbstractSingleEntityEndpoint<EntityType extends Entity, Dt
return normalizedValidationErrors;
}
/**
* Merges the validation errors into the specified map, recording the corresponding node identifier.
*
* @param validationErrorMap map
* @param nodeId id
* @param nodeValidationErrors errors
*/
protected void mergeValidationErrors(final Map<String, Set<NodeIdentifier>> validationErrorMap, final NodeIdentifier nodeId, final Collection<String> nodeValidationErrors) {
if (nodeValidationErrors != null) {
nodeValidationErrors.stream().forEach(
err -> validationErrorMap.computeIfAbsent(err, k -> new HashSet<NodeIdentifier>())
.add(nodeId));
}
}
/**
* @return the class that represents the type of Entity that is expected by this response mapper
*/
protected abstract Class<EntityType> getEntityClass();
/**
* Extracts the DTO from the given entity
* Merges the responses from all nodes in the given map into the single given entity
*
* @param entity the entity to extract the DTO from
* @return the DTO from the given entity
*/
protected abstract DtoType getDto(EntityType entity);
/**
* Merges the responses from all nodes in the given map into the single given DTO
*
* @param clientDto the DTO to merge responses into
* @param dtoMap the responses from all nodes
* @param clientEntity the Entity to merge responses into
* @param entityMap the responses from all nodes
* @param successfulResponses the responses from nodes that completed the request successfully
* @param problematicResponses the responses from nodes that did not complete the request successfully
*/
protected abstract void mergeResponses(DtoType clientDto, Map<NodeIdentifier, DtoType> dtoMap, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses);
protected abstract void mergeResponses(EntityType clientEntity, Map<NodeIdentifier, EntityType> entityMap, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses);
}

View File

@ -32,7 +32,7 @@ import org.apache.nifi.web.api.dto.BulletinBoardDTO;
import org.apache.nifi.web.api.dto.BulletinDTO;
import org.apache.nifi.web.api.entity.BulletinBoardEntity;
public class BulletinBoardEndpointMerger extends AbstractSingleEntityEndpoint<BulletinBoardEntity, BulletinBoardDTO> {
public class BulletinBoardEndpointMerger extends AbstractSingleDTOEndpoint<BulletinBoardEntity, BulletinBoardDTO> {
public static final Pattern BULLETIN_BOARD_URI_PATTERN = Pattern.compile("/nifi-api/flow/bulletin-board");
@Override

View File

@ -33,7 +33,7 @@ import org.apache.nifi.web.api.dto.StateEntryDTO;
import org.apache.nifi.web.api.dto.StateMapDTO;
import org.apache.nifi.web.api.entity.ComponentStateEntity;
public class ComponentStateEndpointMerger extends AbstractSingleEntityEndpoint<ComponentStateEntity, ComponentStateDTO> {
public class ComponentStateEndpointMerger extends AbstractSingleDTOEndpoint<ComponentStateEntity, ComponentStateDTO> {
public static final Pattern PROCESSOR_STATE_URI_PATTERN = Pattern.compile("/nifi-api/processors/[a-f0-9\\-]{36}/state");
public static final Pattern CONTROLLER_SERVICE_STATE_URI_PATTERN = Pattern.compile("/nifi-api/controller-services/node/[a-f0-9\\-]{36}/state");
public static final Pattern REPORTING_TASK_STATE_URI_PATTERN = Pattern.compile("/nifi-api/reporting-tasks/node/[a-f0-9\\-]{36}/state");

View File

@ -31,7 +31,7 @@ import java.util.Map;
import java.util.Set;
import java.util.regex.Pattern;
public class ControllerServiceEndpointMerger extends AbstractSingleEntityEndpoint<ControllerServiceEntity, ControllerServiceDTO> {
public class ControllerServiceEndpointMerger extends AbstractSingleDTOEndpoint<ControllerServiceEntity, ControllerServiceDTO> {
public static final String CONTROLLER_SERVICES_URI = "/nifi-api/controller-services/node";
public static final Pattern CONTROLLER_SERVICE_URI_PATTERN = Pattern.compile("/nifi-api/controller-services/node/[a-f0-9\\-]{36}");

View File

@ -29,7 +29,7 @@ import org.apache.nifi.web.api.dto.BulletinDTO;
import org.apache.nifi.web.api.dto.status.ControllerStatusDTO;
import org.apache.nifi.web.api.entity.ControllerStatusEntity;
public class ControllerStatusEndpointMerger extends AbstractSingleEntityEndpoint<ControllerStatusEntity, ControllerStatusDTO> {
public class ControllerStatusEndpointMerger extends AbstractSingleDTOEndpoint<ControllerStatusEntity, ControllerStatusDTO> {
public static final Pattern CONTROLLER_STATUS_URI_PATTERN = Pattern.compile("/nifi-api/flow/status");
@Override
@ -71,13 +71,6 @@ public class ControllerStatusEndpointMerger extends AbstractSingleEntityEndpoint
StatusMerger.merge(mergedStatus, nodeStatus);
}
final int totalNodeCount = successfulResponses.size() + problematicResponses.size();
final int connectedNodeCount = successfulResponses.size(); // all nodes that responded successfully must be connected. Those that did not will be disconnected.
mergedStatus.setConnectedNodeCount(connectedNodeCount);
mergedStatus.setTotalNodeCount(totalNodeCount);
StatusMerger.updatePrettyPrintedFields(mergedStatus);
}
}

View File

@ -29,7 +29,7 @@ import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.web.api.dto.DropRequestDTO;
import org.apache.nifi.web.api.entity.DropRequestEntity;
public class DropRequestEndpiontMerger extends AbstractSingleEntityEndpoint<DropRequestEntity, DropRequestDTO> {
public class DropRequestEndpiontMerger extends AbstractSingleDTOEndpoint<DropRequestEntity, DropRequestDTO> {
public static final Pattern DROP_REQUESTS_URI = Pattern.compile("/nifi-api/flowfile-queues/[a-f0-9\\-]{36}/drop-requests");
public static final Pattern DROP_REQUEST_URI = Pattern.compile("/nifi-api/flowfile-queues/[a-f0-9\\-]{36}/drop-requests/[a-f0-9\\-]{36}");

View File

@ -0,0 +1,267 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.coordination.http.endpoints;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.manager.StatusMerger;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.web.api.dto.flow.FlowDTO;
import org.apache.nifi.web.api.dto.flow.ProcessGroupFlowDTO;
import org.apache.nifi.web.api.dto.status.ConnectionStatusDTO;
import org.apache.nifi.web.api.dto.status.PortStatusDTO;
import org.apache.nifi.web.api.dto.status.ProcessGroupStatusDTO;
import org.apache.nifi.web.api.dto.status.ProcessorStatusDTO;
import org.apache.nifi.web.api.dto.status.RemoteProcessGroupStatusDTO;
import org.apache.nifi.web.api.entity.ConnectionEntity;
import org.apache.nifi.web.api.entity.FunnelEntity;
import org.apache.nifi.web.api.entity.LabelEntity;
import org.apache.nifi.web.api.entity.PortEntity;
import org.apache.nifi.web.api.entity.ProcessGroupEntity;
import org.apache.nifi.web.api.entity.ProcessGroupFlowEntity;
import org.apache.nifi.web.api.entity.ProcessorEntity;
import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
import java.net.URI;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.regex.Pattern;
public class FlowMerger extends AbstractSingleDTOEndpoint<ProcessGroupFlowEntity, ProcessGroupFlowDTO> {
public static final Pattern FLOW_URI_PATTERN = Pattern.compile("/nifi-api/flow/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))");
@Override
public boolean canHandle(final URI uri, final String method) {
return "GET".equalsIgnoreCase(method) && FLOW_URI_PATTERN.matcher(uri.getPath()).matches();
}
@Override
protected Class<ProcessGroupFlowEntity> getEntityClass() {
return ProcessGroupFlowEntity.class;
}
@Override
protected ProcessGroupFlowDTO getDto(final ProcessGroupFlowEntity entity) {
return entity.getProcessGroupFlow();
}
@Override
protected void mergeResponses(final ProcessGroupFlowDTO clientDto, final Map<NodeIdentifier, ProcessGroupFlowDTO> dtoMap,
final Set<NodeResponse> successfulResponses, final Set<NodeResponse> problematicResponses) {
final FlowDTO flowDto = clientDto.getFlow();
final Map<String, List<ConnectionEntity>> connections = new HashMap<>();
final Map<String, List<FunnelEntity>> funnels = new HashMap<>();
final Map<String, List<PortEntity>> inputPorts = new HashMap<>();
final Map<String, List<LabelEntity>> labels = new HashMap<>();
final Map<String, List<PortEntity>> outputPorts = new HashMap<>();
final Map<String, List<ProcessorEntity>> processors = new HashMap<>();
final Map<String, List<RemoteProcessGroupEntity>> rpgs = new HashMap<>();
final Map<String, List<ProcessGroupEntity>> processGroups = new HashMap<>();
// Create mapping of ComponentID -> all components with that ID (one per node)
for (final ProcessGroupFlowDTO nodeGroupFlowDto : dtoMap.values()) {
final FlowDTO nodeFlowDto = nodeGroupFlowDto.getFlow();
// Merge connection statuses
for (final ConnectionEntity entity : nodeFlowDto.getConnections()) {
connections.computeIfAbsent(entity.getId(), id -> new ArrayList<>()).add(entity);
}
for (final FunnelEntity entity : nodeFlowDto.getFunnels()) {
funnels.computeIfAbsent(entity.getId(), id -> new ArrayList<>()).add(entity);
}
for (final PortEntity entity : nodeFlowDto.getInputPorts()) {
inputPorts.computeIfAbsent(entity.getId(), id -> new ArrayList<>()).add(entity);
}
for (final PortEntity entity : nodeFlowDto.getOutputPorts()) {
outputPorts.computeIfAbsent(entity.getId(), id -> new ArrayList<>()).add(entity);
}
for (final LabelEntity entity : nodeFlowDto.getLabels()) {
labels.computeIfAbsent(entity.getId(), id -> new ArrayList<>()).add(entity);
}
for (final ProcessorEntity entity : nodeFlowDto.getProcessors()) {
processors.computeIfAbsent(entity.getId(), id -> new ArrayList<>()).add(entity);
}
for (final RemoteProcessGroupEntity entity : nodeFlowDto.getRemoteProcessGroups()) {
rpgs.computeIfAbsent(entity.getId(), id -> new ArrayList<>()).add(entity);
}
for (final ProcessGroupEntity entity : nodeFlowDto.getProcessGroups()) {
processGroups.computeIfAbsent(entity.getId(), id -> new ArrayList<>()).add(entity);
}
}
//
// Merge the components that are grouped together by ID
//
// Merge connections
final Set<ConnectionEntity> mergedConnections = new HashSet<>();
for (final List<ConnectionEntity> connectionList : connections.values()) {
mergedConnections.add(mergeConnections(connectionList));
}
flowDto.setConnections(mergedConnections);
// Merge funnel statuses
final Set<FunnelEntity> mergedFunnels = new HashSet<>();
for (final List<FunnelEntity> funnelList : funnels.values()) {
mergedFunnels.add(mergeFunnels(funnelList));
}
flowDto.setFunnels(mergedFunnels);
// Merge input ports
final Set<PortEntity> mergedInputPorts = new HashSet<>();
for (final List<PortEntity> portList : inputPorts.values()) {
mergedInputPorts.add(mergePorts(portList));
}
flowDto.setInputPorts(mergedInputPorts);
// Merge output ports
final Set<PortEntity> mergedOutputPorts = new HashSet<>();
for (final List<PortEntity> portList : outputPorts.values()) {
mergedOutputPorts.add(mergePorts(portList));
}
flowDto.setOutputPorts(mergedOutputPorts);
// Merge labels
final Set<LabelEntity> mergedLabels = new HashSet<>();
for (final List<LabelEntity> labelList : labels.values()) {
mergedLabels.add(mergeLabels(labelList));
}
flowDto.setLabels(mergedLabels);
// Merge processors
final Set<ProcessorEntity> mergedProcessors = new HashSet<>();
for (final List<ProcessorEntity> processorList : processors.values()) {
mergedProcessors.add(mergeProcessors(processorList));
}
flowDto.setProcessors(mergedProcessors);
// Merge Remote Process Groups
final Set<RemoteProcessGroupEntity> mergedRpgs = new HashSet<>();
for (final List<RemoteProcessGroupEntity> rpgList : rpgs.values()) {
mergedRpgs.add(mergeRemoteProcessGroups(rpgList));
}
flowDto.setRemoteProcessGroups(mergedRpgs);
// Merge Process Groups
final Set<ProcessGroupEntity> mergedGroups = new HashSet<>();
for (final List<ProcessGroupEntity> groupList : processGroups.values()) {
mergedGroups.add(mergeProcessGroups(groupList));
}
flowDto.setProcessGroups(mergedGroups);
}
private ConnectionEntity mergeConnections(final List<ConnectionEntity> connections) {
final ConnectionEntity merged = connections.get(0);
final ConnectionStatusDTO statusDto = merged.getStatus();
statusDto.setNodeSnapshots(null);
for (final ConnectionEntity entity : connections) {
if (entity != merged) {
StatusMerger.merge(merged.getStatus().getAggregateSnapshot(), entity.getStatus().getAggregateSnapshot());
}
}
return merged;
}
private PortEntity mergePorts(final List<PortEntity> ports) {
final PortEntity merged = ports.get(0);
final PortStatusDTO statusDto = merged.getStatus();
statusDto.setNodeSnapshots(null);
for (final PortEntity entity : ports) {
if (entity != merged) {
StatusMerger.merge(merged.getStatus().getAggregateSnapshot(), entity.getStatus().getAggregateSnapshot());
}
}
return merged;
}
private FunnelEntity mergeFunnels(final List<FunnelEntity> funnels) {
return funnels.get(0);
}
private LabelEntity mergeLabels(final List<LabelEntity> labels) {
return labels.get(0);
}
private ProcessorEntity mergeProcessors(final List<ProcessorEntity> processors) {
final ProcessorEntity merged = processors.get(0);
final ProcessorStatusDTO statusDto = merged.getStatus();
statusDto.setNodeSnapshots(null);
for (final ProcessorEntity entity : processors) {
if (entity != merged) {
StatusMerger.merge(merged.getStatus().getAggregateSnapshot(), entity.getStatus().getAggregateSnapshot());
}
}
return merged;
}
private RemoteProcessGroupEntity mergeRemoteProcessGroups(final List<RemoteProcessGroupEntity> rpgs) {
final RemoteProcessGroupEntity merged = rpgs.get(0);
final RemoteProcessGroupStatusDTO statusDto = merged.getStatus();
statusDto.setNodeSnapshots(null);
for (final RemoteProcessGroupEntity entity : rpgs) {
if (entity != merged) {
StatusMerger.merge(merged.getStatus().getAggregateSnapshot(), entity.getStatus().getAggregateSnapshot());
}
}
return merged;
}
private ProcessGroupEntity mergeProcessGroups(final List<ProcessGroupEntity> groups) {
final ProcessGroupEntity merged = groups.get(0);
final ProcessGroupStatusDTO statusDto = merged.getStatus();
statusDto.setNodeSnapshots(null);
for (final ProcessGroupEntity entity : groups) {
if (entity != merged) {
StatusMerger.merge(merged.getStatus().getAggregateSnapshot(), entity.getStatus().getAggregateSnapshot());
}
}
// We merge only the statuses of the Process Groups. The child components are not
// necessary for a FlowProcessGroupDTO, so we just ensure that they are null
if (merged.getComponent() != null) {
merged.getComponent().setContents(null);
}
return merged;
}
}

View File

@ -57,22 +57,12 @@ public class FlowSnippetEndpointMerger implements EndpointResponseMerger {
final FlowDTO nodeContents = nodeResponseEntity.getFlow();
for (final ProcessorEntity nodeProcessor : nodeContents.getProcessors()) {
Map<NodeIdentifier, ProcessorEntity> innerMap = processorMap.get(nodeProcessor.getId());
if (innerMap == null) {
innerMap = new HashMap<>();
processorMap.put(nodeProcessor.getId(), innerMap);
}
Map<NodeIdentifier, ProcessorEntity> innerMap = processorMap.computeIfAbsent(nodeProcessor.getId(), id -> new HashMap<>());
innerMap.put(nodeResponse.getNodeId(), nodeProcessor);
}
for (final RemoteProcessGroupEntity nodeRemoteProcessGroup : nodeContents.getRemoteProcessGroups()) {
Map<NodeIdentifier, RemoteProcessGroupEntity> innerMap = remoteProcessGroupMap.get(nodeRemoteProcessGroup.getId());
if (innerMap == null) {
innerMap = new HashMap<>();
remoteProcessGroupMap.put(nodeRemoteProcessGroup.getId(), innerMap);
}
Map<NodeIdentifier, RemoteProcessGroupEntity> innerMap = remoteProcessGroupMap.computeIfAbsent(nodeRemoteProcessGroup.getId(), id -> new HashMap<>());
innerMap.put(nodeResponse.getNodeId(), nodeRemoteProcessGroup);
}
}

View File

@ -35,7 +35,7 @@ import org.apache.nifi.web.api.dto.ListingRequestDTO;
import org.apache.nifi.web.api.dto.QueueSizeDTO;
import org.apache.nifi.web.api.entity.ListingRequestEntity;
public class ListFlowFilesEndpointMerger extends AbstractSingleEntityEndpoint<ListingRequestEntity, ListingRequestDTO> {
public class ListFlowFilesEndpointMerger extends AbstractSingleDTOEndpoint<ListingRequestEntity, ListingRequestDTO> {
public static final Pattern LISTING_REQUESTS_URI = Pattern.compile("/nifi-api/flowfile-queues/[a-f0-9\\-]{36}/listing-requests");
public static final Pattern LISTING_REQUEST_URI = Pattern.compile("/nifi-api/flowfile-queues/[a-f0-9\\-]{36}/listing-requests/[a-f0-9\\-]{36}");

View File

@ -25,11 +25,12 @@ import java.util.regex.Pattern;
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.manager.StatusMerger;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.web.api.dto.ProcessorDTO;
import org.apache.nifi.web.api.entity.ProcessorEntity;
public class ProcessorEndpointMerger extends AbstractSingleEntityEndpoint<ProcessorEntity, ProcessorDTO> implements EndpointResponseMerger {
public class ProcessorEndpointMerger extends AbstractSingleEntityEndpoint<ProcessorEntity> implements EndpointResponseMerger {
public static final Pattern PROCESSORS_URI_PATTERN = Pattern.compile("/nifi-api/processors");
public static final Pattern PROCESSOR_URI_PATTERN = Pattern.compile("/nifi-api/processors/[a-f0-9\\-]{36}");
public static final Pattern CLUSTER_PROCESSOR_URI_PATTERN = Pattern.compile("/nifi-api/cluster/processors/[a-f0-9\\-]{36}");
@ -51,12 +52,7 @@ public class ProcessorEndpointMerger extends AbstractSingleEntityEndpoint<Proces
return ProcessorEntity.class;
}
@Override
protected ProcessorDTO getDto(final ProcessorEntity entity) {
return entity.getComponent();
}
@Override
protected void mergeResponses(final ProcessorDTO clientDto, final Map<NodeIdentifier, ProcessorDTO> dtoMap, final Set<NodeResponse> successfulResponses,
final Set<NodeResponse> problematicResponses) {
final Map<String, Set<NodeIdentifier>> validationErrorMap = new HashMap<>();
@ -73,6 +69,7 @@ public class ProcessorEndpointMerger extends AbstractSingleEntityEndpoint<Proces
clientDto.setValidationErrors(normalizedMergedValidationErrors(validationErrorMap, dtoMap.size()));
}
@Override
protected void mergeResponses(final ProcessorEntity clientEntity, final Map<NodeIdentifier, ProcessorEntity> entityMap, final Set<NodeResponse> successfulResponses,
final Set<NodeResponse> problematicResponses) {
@ -84,6 +81,14 @@ public class ProcessorEndpointMerger extends AbstractSingleEntityEndpoint<Proces
dtoMap.put(entry.getKey(), nodeProcDto);
}
for (final Map.Entry<NodeIdentifier, ProcessorEntity> entry : entityMap.entrySet()) {
final NodeIdentifier nodeId = entry.getKey();
final ProcessorEntity entity = entry.getValue();
if (entity != clientEntity) {
StatusMerger.merge(clientEntity.getStatus(), entity.getStatus(), nodeId.getId(), nodeId.getApiAddress(), nodeId.getApiPort());
}
}
mergeResponses(clientDto, dtoMap, successfulResponses, problematicResponses);
}
}

View File

@ -27,7 +27,7 @@ import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.web.api.dto.provenance.ProvenanceEventDTO;
import org.apache.nifi.web.api.entity.ProvenanceEventEntity;
public class ProvenanceEventEndpointMerger extends AbstractSingleEntityEndpoint<ProvenanceEventEntity, ProvenanceEventDTO> {
public class ProvenanceEventEndpointMerger extends AbstractSingleDTOEndpoint<ProvenanceEventEntity, ProvenanceEventDTO> {
public static final Pattern PROVENANCE_EVENT_URI = Pattern.compile("/nifi-api/provenance/events/[0-9]+");
@Override

View File

@ -33,7 +33,7 @@ import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO;
import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
public class RemoteProcessGroupEndpointMerger extends AbstractSingleEntityEndpoint<RemoteProcessGroupEntity, RemoteProcessGroupDTO> {
public class RemoteProcessGroupEndpointMerger extends AbstractSingleDTOEndpoint<RemoteProcessGroupEntity, RemoteProcessGroupDTO> {
public static final Pattern REMOTE_PROCESS_GROUPS_URI_PATTERN = Pattern.compile("/nifi-api/remote-process-groups");
public static final Pattern REMOTE_PROCESS_GROUP_URI_PATTERN = Pattern.compile("/nifi-api/remote-process-groups/[a-f0-9\\-]{36}");

View File

@ -28,7 +28,7 @@ import java.util.Map;
import java.util.Set;
import java.util.regex.Pattern;
public class ReportingTaskEndpointMerger extends AbstractSingleEntityEndpoint<ReportingTaskEntity, ReportingTaskDTO> {
public class ReportingTaskEndpointMerger extends AbstractSingleDTOEndpoint<ReportingTaskEntity, ReportingTaskDTO> {
public static final String REPORTING_TASKS_URI = "/nifi-api/controller/reporting-tasks/node";
public static final Pattern REPORTING_TASK_URI_PATTERN = Pattern.compile("/nifi-api/reporting-tasks/node/[a-f0-9\\-]{36}");

View File

@ -27,6 +27,7 @@ public interface RequestReplicator {
public static final String REQUEST_TRANSACTION_ID_HEADER = "X-RequestTransactionId";
public static final String CLUSTER_ID_GENERATION_SEED_HEADER = "X-Cluster-Id-Generation-Seed";
public static final String REPLICATION_INDICATOR_HEADER = "X-Request-Replicated";
/**
* The HTTP header that the requestor specifies to ask a node if they are able to process a given request. The value
@ -37,21 +38,30 @@ public interface RequestReplicator {
public static final String NODE_CONTINUE = "150-NodeContinue";
public static final int NODE_CONTINUE_STATUS_CODE = 150;
/**
* Starts the instance for replicating requests. Calling this method on an already started instance has no effect.
*/
void start();
public static final String CLAIM_CANCEL_HEADER = "X-Cancel-Claim";
/**
* Stops the instance from replicating requests. Calling this method on a stopped instance has no effect.
*/
void stop();
void shutdown();
/**
* @return true if the instance is started; false otherwise.
* Replicates a request to each node in the cluster. If the request attempts to modify the flow and there is a node
* that is not currently connected, an Exception will be thrown. Otherwise, the returned AsyncClusterResponse object
* will contain the results that are immediately available, as well as an identifier for obtaining an updated result
* later.
*
* @param method the HTTP method (e.g., POST, PUT)
* @param uri the base request URI (up to, but not including, the query string)
* @param entity an entity
* @param headers any HTTP headers
* @return an AsyncClusterResponse that indicates the current status of the request and provides an identifier for obtaining an updated response later
*
* @throws ConnectingNodeMutableRequestException if the request attempts to modify the flow and there is a node that is in the CONNECTING state
* @throws DisconnectedNodeMutableRequestException if the request attempts to modify the flow and there is a node that is in the DISCONNECTED state
*/
boolean isRunning();
AsyncClusterResponse replicate(String method, URI uri, Object entity, Map<String, String> headers);
/**
* Requests are sent to each node in the given set of Node Identifiers. The returned AsyncClusterResponse object will contain

View File

@ -47,14 +47,14 @@ import org.apache.nifi.cluster.coordination.ClusterCoordinator;
import org.apache.nifi.cluster.coordination.http.HttpResponseMerger;
import org.apache.nifi.cluster.coordination.http.StandardHttpResponseMerger;
import org.apache.nifi.cluster.coordination.node.NodeConnectionState;
import org.apache.nifi.cluster.flow.DataFlowManagementService;
import org.apache.nifi.cluster.flow.PersistedFlowState;
import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.manager.exception.ConnectingNodeMutableRequestException;
import org.apache.nifi.cluster.manager.exception.DisconnectedNodeMutableRequestException;
import org.apache.nifi.cluster.manager.exception.IllegalClusterStateException;
import org.apache.nifi.cluster.manager.exception.NoConnectedNodesException;
import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
import org.apache.nifi.cluster.manager.exception.UriConstructionException;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.events.EventReporter;
import org.apache.nifi.logging.NiFiLog;
@ -76,14 +76,12 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
private static final int MAX_CONCURRENT_REQUESTS = 100;
private final Client client; // the client to use for issuing requests
private final int numThreads; // number of threads to use for request replication
private final int connectionTimeoutMs; // connection timeout per node request
private final int readTimeoutMs; // read timeout per node request
private final HttpResponseMerger responseMerger;
private final EventReporter eventReporter;
private final RequestCompletionCallback callback;
private final ClusterCoordinator clusterCoordinator;
private final DataFlowManagementService dfmService;
private ExecutorService executorService;
private ScheduledExecutorService maintenanceExecutor;
@ -101,8 +99,8 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
* @param eventReporter an EventReporter that can be used to notify users of interesting events. May be null.
*/
public ThreadPoolRequestReplicator(final int numThreads, final Client client, final ClusterCoordinator clusterCoordinator,
final RequestCompletionCallback callback, final EventReporter eventReporter, final DataFlowManagementService dfmService) {
this(numThreads, client, clusterCoordinator, "3 sec", "3 sec", callback, eventReporter, null, dfmService);
final RequestCompletionCallback callback, final EventReporter eventReporter) {
this(numThreads, client, clusterCoordinator, "5 sec", "5 sec", callback, eventReporter);
}
/**
@ -117,36 +115,33 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
* @param eventReporter an EventReporter that can be used to notify users of interesting events. May be null.
*/
public ThreadPoolRequestReplicator(final int numThreads, final Client client, final ClusterCoordinator clusterCoordinator,
final String connectionTimeout, final String readTimeout, final RequestCompletionCallback callback, final EventReporter eventReporter,
final WebClusterManager clusterManager, final DataFlowManagementService dfmService) {
final String connectionTimeout, final String readTimeout, final RequestCompletionCallback callback, final EventReporter eventReporter) {
if (numThreads <= 0) {
throw new IllegalArgumentException("The number of threads must be greater than zero.");
} else if (client == null) {
throw new IllegalArgumentException("Client may not be null.");
}
this.numThreads = numThreads;
this.client = client;
this.clusterCoordinator = clusterCoordinator;
this.connectionTimeoutMs = (int) FormatUtils.getTimeDuration(connectionTimeout, TimeUnit.MILLISECONDS);
this.readTimeoutMs = (int) FormatUtils.getTimeDuration(readTimeout, TimeUnit.MILLISECONDS);
this.responseMerger = new StandardHttpResponseMerger(clusterManager);
this.responseMerger = new StandardHttpResponseMerger();
this.eventReporter = eventReporter;
this.callback = callback;
this.dfmService = dfmService;
client.getProperties().put(ClientConfig.PROPERTY_CONNECT_TIMEOUT, connectionTimeoutMs);
client.getProperties().put(ClientConfig.PROPERTY_READ_TIMEOUT, readTimeoutMs);
client.getProperties().put(ClientConfig.PROPERTY_FOLLOW_REDIRECTS, Boolean.TRUE);
}
@Override
public void start() {
if (isRunning()) {
return;
}
final AtomicInteger threadId = new AtomicInteger(0);
executorService = Executors.newFixedThreadPool(numThreads, r -> {
final Thread t = Executors.defaultThreadFactory().newThread(r);
t.setDaemon(true);
t.setName("Replicate Request Thread-" + threadId.incrementAndGet());
return t;
});
executorService = Executors.newFixedThreadPool(numThreads);
maintenanceExecutor = Executors.newScheduledThreadPool(1, new ThreadFactory() {
@Override
public Thread newThread(final Runnable r) {
@ -161,25 +156,61 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
}
@Override
public boolean isRunning() {
return executorService != null && !executorService.isShutdown();
}
@Override
public void stop() {
if (!isRunning()) {
return;
}
public void shutdown() {
executorService.shutdown();
maintenanceExecutor.shutdown();
}
@Override
public AsyncClusterResponse replicate(String method, URI uri, Object entity, Map<String, String> headers) {
final Map<NodeConnectionState, List<NodeIdentifier>> stateMap = clusterCoordinator.getConnectionStates();
final boolean mutable = isMutableRequest(method, uri.getPath());
// If the request is mutable, ensure that all nodes are connected.
if (mutable) {
final List<NodeIdentifier> disconnected = stateMap.get(NodeConnectionState.DISCONNECTED);
if (disconnected != null && !disconnected.isEmpty()) {
if (disconnected.size() == 1) {
throw new DisconnectedNodeMutableRequestException("Node " + disconnected.iterator().next() + " is currently disconnected");
} else {
throw new DisconnectedNodeMutableRequestException(disconnected.size() + " Nodes are currently disconnected");
}
}
final List<NodeIdentifier> disconnecting = stateMap.get(NodeConnectionState.DISCONNECTING);
if (disconnecting != null && !disconnecting.isEmpty()) {
if (disconnecting.size() == 1) {
throw new DisconnectedNodeMutableRequestException("Node " + disconnecting.iterator().next() + " is currently disconnecting");
} else {
throw new DisconnectedNodeMutableRequestException(disconnecting.size() + " Nodes are currently disconnecting");
}
}
final List<NodeIdentifier> connecting = stateMap.get(NodeConnectionState.CONNECTING);
if (connecting != null && !connecting.isEmpty()) {
if (connecting.size() == 1) {
throw new ConnectingNodeMutableRequestException("Node " + connecting.iterator().next() + " is currently connecting");
} else {
throw new ConnectingNodeMutableRequestException(connecting.size() + " Nodes are currently connecting");
}
}
}
final List<NodeIdentifier> nodeIds = stateMap.get(NodeConnectionState.CONNECTED);
if (nodeIds == null || nodeIds.isEmpty()) {
throw new NoConnectedNodesException();
}
final Set<NodeIdentifier> nodeIdSet = new HashSet<>(nodeIds);
return replicate(nodeIdSet, method, uri, entity, headers);
}
@Override
public AsyncClusterResponse replicate(Set<NodeIdentifier> nodeIds, String method, URI uri, Object entity, Map<String, String> headers) {
final Map<String, String> headersPlusIdGenerationSeed = new HashMap<>(headers);
headersPlusIdGenerationSeed.put(RequestReplicator.CLUSTER_ID_GENERATION_SEED_HEADER, UUID.randomUUID().toString());
return replicate(nodeIds, method, uri, entity, headersPlusIdGenerationSeed, true, null);
final Map<String, String> updatedHeaders = new HashMap<>(headers);
updatedHeaders.put(RequestReplicator.CLUSTER_ID_GENERATION_SEED_HEADER, UUID.randomUUID().toString());
updatedHeaders.put(RequestReplicator.REPLICATION_INDICATOR_HEADER, "true");
return replicate(nodeIds, method, uri, entity, updatedHeaders, true, null);
}
/**
@ -209,6 +240,18 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
throw new IllegalArgumentException("Cannot replicate request to 0 nodes");
}
// verify all of the nodes exist and are in the proper state
for (final NodeIdentifier nodeId : nodeIds) {
final NodeConnectionStatus status = clusterCoordinator.getConnectionStatus(nodeId);
if (status == null) {
throw new UnknownNodeException("Node " + nodeId + " does not exist in this cluster");
}
if (status.getState() != NodeConnectionState.CONNECTED) {
throw new IllegalClusterStateException("Cannot replicate request to Node " + nodeId + " because the node is not connected");
}
}
logger.debug("Replicating request {} {} with entity {} to {}; response is {}", method, uri, entity, nodeIds, response);
// Update headers to indicate the current revision so that we can
@ -238,7 +281,6 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
}
logger.debug("For Request ID {}, response object is {}", requestId, response);
// setRevision(updatedHeaders);
// if mutable request, we have to do a two-phase commit where we ask each node to verify
// that the request can take place and then, if all nodes agree that it can, we can actually
@ -259,7 +301,6 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
finalResponse.add(nodeResponse);
};
// replicate the request to all nodes
final Function<NodeIdentifier, NodeHttpRequest> requestFactory =
nodeId -> new NodeHttpRequest(nodeId, method, createURI(uri, nodeId), entity, updatedHeaders, nodeCompletionCallback);
@ -309,6 +350,21 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
return;
}
final Thread cancelClaimThread = new Thread(new Runnable() {
@Override
public void run() {
logger.debug("Found {} dissenting nodes for {} {}; canceling claim request", dissentingCount, method, uri.getPath());
updatedHeaders.put(CLAIM_CANCEL_HEADER, "true");
final Function<NodeIdentifier, NodeHttpRequest> requestFactory =
nodeId -> new NodeHttpRequest(nodeId, method, createURI(uri, nodeId), entity, updatedHeaders, null);
replicateRequest(nodeIds, uri.getScheme(), uri.getPath(), requestFactory, updatedHeaders);
}
});
cancelClaimThread.setName("Cancel Claims");
cancelClaimThread.start();
// Add a NodeResponse for each node to the Cluster Response
// Check that all nodes responded successfully.
for (final NodeResponse response : nodeResponses) {
@ -354,9 +410,6 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
}
};
// notify dataflow management service that flow state is not known
dfmService.setPersistedFlowState(PersistedFlowState.UNKNOWN);
// Callback function for generating a NodeHttpRequestCallable that can be used to perform the work
final Function<NodeIdentifier, NodeHttpRequest> requestFactory = nodeId -> new NodeHttpRequest(nodeId, method, createURI(uri, nodeId), entity, updatedHeaders, completionCallback);
@ -449,11 +502,7 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
* @param requestId the ID of the request that has been consumed by the client
*/
private void onResponseConsumed(final String requestId) {
final AsyncClusterResponse response = responseMap.remove(requestId);
if (response != null && logger.isDebugEnabled()) {
logTimingInfo(response);
}
responseMap.remove(requestId);
}
/**
@ -466,12 +515,6 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
private void onCompletedResponse(final String requestId) {
final AsyncClusterResponse response = responseMap.get(requestId);
if (response != null) {
if (isMutableRequest(response.getMethod(), response.getURIPath())) {
dfmService.setPersistedFlowState(PersistedFlowState.STALE);
}
}
if (response != null && callback != null) {
try {
callback.afterRequest(response.getURIPath(), response.getMethod(), response.getCompletedNodeResponses());
@ -482,6 +525,10 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
}
}
if (response != null && logger.isDebugEnabled()) {
logTimingInfo(response);
}
// If we have any nodes that are slow to respond, keep track of this. If the same node is slow 3 times in
// a row, log a warning to indicate that the node is responding slowly.
final Set<NodeIdentifier> slowResponseNodes = ResponseUtils.findLongResponseTimes(response, 1.5D);

View File

@ -0,0 +1,116 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.coordination.node;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.nio.charset.StandardCharsets;
import org.apache.curator.RetryPolicy;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.CuratorFrameworkFactory;
import org.apache.curator.retry.RetryNTimes;
import org.apache.nifi.cluster.protocol.AbstractNodeProtocolSender;
import org.apache.nifi.cluster.protocol.ProtocolContext;
import org.apache.nifi.cluster.protocol.ProtocolException;
import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
import org.apache.nifi.controller.cluster.ZooKeeperClientConfig;
import org.apache.nifi.io.socket.SocketConfiguration;
import org.apache.nifi.util.NiFiProperties;
import org.apache.zookeeper.KeeperException.NoNodeException;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Uses Apache Curator to determine the address of the current cluster coordinator
*/
public class CuratorNodeProtocolSender extends AbstractNodeProtocolSender {
private static final Logger logger = LoggerFactory.getLogger(CuratorNodeProtocolSender.class);
private final String coordinatorPath;
private final ZooKeeperClientConfig zkConfig;
private InetSocketAddress coordinatorAddress;
public CuratorNodeProtocolSender(final SocketConfiguration socketConfig, final ProtocolContext<ProtocolMessage> protocolContext, final NiFiProperties properties) {
super(socketConfig, protocolContext);
zkConfig = ZooKeeperClientConfig.createConfig(properties);
coordinatorPath = zkConfig.resolvePath("cluster/nodes/coordinator");
}
@Override
protected synchronized InetSocketAddress getServiceAddress() throws IOException {
if (coordinatorAddress != null) {
return coordinatorAddress;
}
final RetryPolicy retryPolicy = new RetryNTimes(0, 0);
final CuratorFramework curatorClient = CuratorFrameworkFactory.newClient(zkConfig.getConnectString(),
zkConfig.getSessionTimeoutMillis(), zkConfig.getConnectionTimeoutMillis(), retryPolicy);
curatorClient.start();
try {
// Get coordinator address and add watcher to change who we are heartbeating to if the value changes.
final byte[] coordinatorAddressBytes = curatorClient.getData().usingWatcher(new Watcher() {
@Override
public void process(final WatchedEvent event) {
coordinatorAddress = null;
}
}).forPath(coordinatorPath);
final String address = new String(coordinatorAddressBytes, StandardCharsets.UTF_8);
logger.info("Determined that Cluster Coordinator is located at {}; will use this address for sending heartbeat messages", address);
final String[] splits = address.split(":");
if (splits.length != 2) {
final String message = String.format("Attempted to determine Cluster Coordinator address. Zookeeper indicates "
+ "that address is %s, but this is not in the expected format of <hostname>:<port>", address);
logger.error(message);
throw new ProtocolException(message);
}
final String hostname = splits[0];
final int port;
try {
port = Integer.parseInt(splits[1]);
if (port < 1 || port > 65535) {
throw new NumberFormatException("Port must be in the range of 1 - 65535 but got " + port);
}
} catch (final NumberFormatException nfe) {
final String message = String.format("Attempted to determine Cluster Coordinator address. Zookeeper indicates "
+ "that address is %s, but the port is not a valid port number", address);
logger.error(message);
throw new ProtocolException(message);
}
final InetSocketAddress socketAddress = InetSocketAddress.createUnresolved(hostname, port);
coordinatorAddress = socketAddress;
return socketAddress;
} catch (final NoNodeException nne) {
logger.info("No node has yet been elected Cluster Coordinator. Cannot establish connection to cluster yet.");
throw new ProtocolException("No node has yet been elected Cluster Coordinator. Cannot establish connection to cluster yet.");
} catch (Exception e) {
throw new IOException("Unable to determine Cluster Coordinator from ZooKeeper", e);
} finally {
curatorClient.close();
}
}
}

View File

@ -0,0 +1,782 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.coordination.node;
import org.apache.commons.collections4.queue.CircularFifoQueue;
import org.apache.nifi.cluster.coordination.ClusterCoordinator;
import org.apache.nifi.cluster.coordination.http.HttpResponseMerger;
import org.apache.nifi.cluster.coordination.http.StandardHttpResponseMerger;
import org.apache.nifi.cluster.coordination.http.replication.RequestCompletionCallback;
import org.apache.nifi.cluster.event.Event;
import org.apache.nifi.cluster.event.NodeEvent;
import org.apache.nifi.cluster.firewall.ClusterNodeFirewall;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.protocol.ComponentRevision;
import org.apache.nifi.cluster.protocol.ConnectionRequest;
import org.apache.nifi.cluster.protocol.ConnectionResponse;
import org.apache.nifi.cluster.protocol.DataFlow;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.cluster.protocol.ProtocolException;
import org.apache.nifi.cluster.protocol.ProtocolHandler;
import org.apache.nifi.cluster.protocol.StandardDataFlow;
import org.apache.nifi.cluster.protocol.impl.ClusterCoordinationProtocolSenderListener;
import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage;
import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage;
import org.apache.nifi.cluster.protocol.message.DisconnectMessage;
import org.apache.nifi.cluster.protocol.message.NodeStatusChangeMessage;
import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType;
import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage;
import org.apache.nifi.events.EventReporter;
import org.apache.nifi.reporting.Severity;
import org.apache.nifi.services.FlowService;
import org.apache.nifi.web.revision.RevisionManager;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandler, RequestCompletionCallback {
private static final Logger logger = LoggerFactory.getLogger(NodeClusterCoordinator.class);
private static final String EVENT_CATEGORY = "Clustering";
private static final Pattern COUNTER_URI_PATTERN = Pattern.compile("/nifi-api/controller/counters/[a-f0-9\\-]{36}");
private final String instanceId = UUID.randomUUID().toString();
private volatile NodeIdentifier nodeId;
private final ClusterCoordinationProtocolSenderListener senderListener;
private final EventReporter eventReporter;
private final ClusterNodeFirewall firewall;
private final RevisionManager revisionManager;
private volatile FlowService flowService;
private volatile boolean connected;
private final ConcurrentMap<NodeIdentifier, NodeConnectionStatus> nodeStatuses = new ConcurrentHashMap<>();
private final ConcurrentMap<NodeIdentifier, CircularFifoQueue<NodeEvent>> nodeEvents = new ConcurrentHashMap<>();
public NodeClusterCoordinator(final ClusterCoordinationProtocolSenderListener senderListener,
final EventReporter eventReporter, final ClusterNodeFirewall firewall, final RevisionManager revisionManager) {
this.senderListener = senderListener;
this.flowService = null;
this.eventReporter = eventReporter;
this.firewall = firewall;
this.revisionManager = revisionManager;
senderListener.addHandler(this);
}
@Override
public void shutdown() {
final NodeConnectionStatus shutdownStatus = new NodeConnectionStatus(getLocalNodeIdentifier(), DisconnectionCode.NODE_SHUTDOWN);
notifyOthersOfNodeStatusChange(shutdownStatus);
logger.info("Successfully notified other nodes that I am shutting down");
}
@Override
public void setLocalNodeIdentifier(final NodeIdentifier nodeId) {
this.nodeId = nodeId;
}
NodeIdentifier getLocalNodeIdentifier() {
return nodeId;
}
@Override
public void resetNodeStatuses(final Map<NodeIdentifier, NodeConnectionStatus> statusMap) {
logger.info("Resetting cluster node statuses from {} to {}", nodeStatuses, statusMap);
// For each proposed replacement, update the nodeStatuses map if and only if the replacement
// has a larger update id than the current value.
for (final Map.Entry<NodeIdentifier, NodeConnectionStatus> entry : statusMap.entrySet()) {
final NodeIdentifier nodeId = entry.getKey();
final NodeConnectionStatus proposedStatus = entry.getValue();
boolean updated = false;
while (!updated) {
final NodeConnectionStatus currentStatus = nodeStatuses.get(nodeId);
if (currentStatus == null || proposedStatus.getUpdateIdentifier() > currentStatus.getUpdateIdentifier()) {
updated = replaceNodeStatus(nodeId, currentStatus, proposedStatus);
} else {
updated = true;
}
}
}
}
/**
* Attempts to update the nodeStatuses map by changing the value for the given node id from the current status to the new status, as in
* ConcurrentMap.replace(nodeId, currentStatus, newStatus) but with the difference that this method can handle a <code>null</code> value
* for currentStatus
*
* @param nodeId the node id
* @param currentStatus the current status, or <code>null</code> if there is no value currently
* @param newStatus the new status to set
* @return <code>true</code> if the map was updated, false otherwise
*/
private boolean replaceNodeStatus(final NodeIdentifier nodeId, final NodeConnectionStatus currentStatus, final NodeConnectionStatus newStatus) {
if (newStatus == null) {
logger.error("Cannot change node status for {} from {} to {} because new status is null", nodeId, currentStatus, newStatus);
logger.error("", new NullPointerException());
}
if (currentStatus == null) {
final NodeConnectionStatus existingValue = nodeStatuses.putIfAbsent(nodeId, newStatus);
return existingValue == null;
}
return nodeStatuses.replace(nodeId, currentStatus, newStatus);
}
@Override
public void requestNodeConnect(final NodeIdentifier nodeId, final String userDn) {
if (userDn == null) {
reportEvent(nodeId, Severity.INFO, "Requesting that node connect to cluster");
} else {
reportEvent(nodeId, Severity.INFO, "Requesting that node connect to cluster on behalf of " + userDn);
}
updateNodeStatus(new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTING, System.currentTimeMillis()));
// create the request
final ReconnectionRequestMessage request = new ReconnectionRequestMessage();
request.setNodeId(nodeId);
request.setInstanceId(instanceId);
requestReconnectionAsynchronously(request, 10, 5);
}
@Override
public void finishNodeConnection(final NodeIdentifier nodeId) {
final NodeConnectionState state = getConnectionState(nodeId);
if (state == null) {
logger.debug("Attempted to finish node connection for {} but node is not known. Requesting that node connect", nodeId);
requestNodeConnect(nodeId, null);
return;
}
if (state == NodeConnectionState.CONNECTED) {
// already connected. Nothing to do.
return;
}
if (state == NodeConnectionState.DISCONNECTED || state == NodeConnectionState.DISCONNECTING) {
logger.debug("Attempted to finish node connection for {} but node state was {}. Requesting that node connect", nodeId, state);
requestNodeConnect(nodeId, null);
return;
}
logger.info("{} is now connected", nodeId);
final boolean updated = updateNodeStatus(new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTED));
if (!updated) {
logger.error("Attempting to Finish Node Connection but could not find Node with Identifier {}", nodeId);
}
}
@Override
public void requestNodeDisconnect(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode, final String explanation) {
logger.info("Requesting that {} disconnect due to {}", nodeId, explanation == null ? disconnectionCode : explanation);
updateNodeStatus(new NodeConnectionStatus(nodeId, disconnectionCode, explanation));
// There is no need to tell the node that it's disconnected if it is due to being
// shutdown, as we will not be able to connect to the node anyway.
if (disconnectionCode == DisconnectionCode.NODE_SHUTDOWN) {
return;
}
final DisconnectMessage request = new DisconnectMessage();
request.setNodeId(nodeId);
request.setExplanation(explanation);
addNodeEvent(nodeId, "Disconnection requested due to " + explanation);
disconnectAsynchronously(request, 10, 5);
}
@Override
public void disconnectionRequestedByNode(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode, final String explanation) {
logger.info("{} requested disconnection from cluster due to {}", nodeId, explanation == null ? disconnectionCode : explanation);
updateNodeStatus(new NodeConnectionStatus(nodeId, disconnectionCode, explanation));
final Severity severity;
switch (disconnectionCode) {
case STARTUP_FAILURE:
case MISMATCHED_FLOWS:
case UNKNOWN:
severity = Severity.ERROR;
break;
default:
severity = Severity.INFO;
break;
}
reportEvent(nodeId, severity, "Node disconnected from cluster due to " + explanation);
}
@Override
public void removeNode(final NodeIdentifier nodeId, final String userDn) {
reportEvent(nodeId, Severity.INFO, "User " + userDn + " requested that node be removed from cluster");
nodeStatuses.remove(nodeId);
nodeEvents.remove(nodeId);
notifyOthersOfNodeStatusChange(new NodeConnectionStatus(nodeId, NodeConnectionState.REMOVED));
}
@Override
public NodeConnectionStatus getConnectionStatus(final NodeIdentifier nodeId) {
return nodeStatuses.get(nodeId);
}
private NodeConnectionState getConnectionState(final NodeIdentifier nodeId) {
final NodeConnectionStatus status = getConnectionStatus(nodeId);
return status == null ? null : status.getState();
}
@Override
public Map<NodeConnectionState, List<NodeIdentifier>> getConnectionStates() {
final Map<NodeConnectionState, List<NodeIdentifier>> connectionStates = new HashMap<>();
for (final Map.Entry<NodeIdentifier, NodeConnectionStatus> entry : nodeStatuses.entrySet()) {
final NodeConnectionState state = entry.getValue().getState();
final List<NodeIdentifier> nodeIds = connectionStates.computeIfAbsent(state, s -> new ArrayList<NodeIdentifier>());
nodeIds.add(entry.getKey());
}
return connectionStates;
}
@Override
public boolean isBlockedByFirewall(final String hostname) {
return firewall != null && !firewall.isPermissible(hostname);
}
@Override
public void reportEvent(final NodeIdentifier nodeId, final Severity severity, final String event) {
eventReporter.reportEvent(severity, nodeId == null ? EVENT_CATEGORY : nodeId.toString(), event);
if (nodeId != null) {
addNodeEvent(nodeId, severity, event);
}
final String message = nodeId == null ? event : "Event Reported for " + nodeId + ": " + event;
switch (severity) {
case ERROR:
logger.error(message);
break;
case WARNING:
logger.warn(message);
break;
case INFO:
logger.info(message);
break;
}
}
@Override
public synchronized void updateNodeRoles(final NodeIdentifier nodeId, final Set<String> roles) {
boolean updated = false;
while (!updated) {
final NodeConnectionStatus currentStatus = nodeStatuses.get(nodeId);
if (currentStatus == null) {
throw new IllegalStateException("Cannot update roles for " + nodeId + " to " + roles + " because the node is not part of this cluster");
}
if (currentStatus.getRoles().equals(roles)) {
logger.debug("Roles for {} already up-to-date as {}", nodeId, roles);
return;
}
final NodeConnectionStatus updatedStatus = new NodeConnectionStatus(currentStatus, roles);
updated = replaceNodeStatus(nodeId, currentStatus, updatedStatus);
if (updated) {
logger.info("Updated Roles of {} from {} to {}", nodeId, currentStatus, updatedStatus);
notifyOthersOfNodeStatusChange(updatedStatus);
}
}
// If any other node contains any of the given roles, revoke the role from the other node.
for (final String role : roles) {
for (final Map.Entry<NodeIdentifier, NodeConnectionStatus> entry : nodeStatuses.entrySet()) {
if (entry.getKey().equals(nodeId)) {
continue;
}
updated = false;
while (!updated) {
final NodeConnectionStatus status = entry.getValue();
if (status.getRoles().contains(role)) {
final Set<String> newRoles = new HashSet<>(status.getRoles());
newRoles.remove(role);
final NodeConnectionStatus updatedStatus = new NodeConnectionStatus(status, newRoles);
updated = replaceNodeStatus(entry.getKey(), status, updatedStatus);
if (updated) {
logger.info("Updated Roles of {} from {} to {}", nodeId, status, updatedStatus);
notifyOthersOfNodeStatusChange(updatedStatus);
}
} else {
updated = true;
}
}
}
}
}
@Override
public NodeIdentifier getNodeIdentifier(final String uuid) {
for (final NodeIdentifier nodeId : nodeStatuses.keySet()) {
if (nodeId.getId().equals(uuid)) {
return nodeId;
}
}
return null;
}
@Override
public Set<NodeIdentifier> getNodeIdentifiers(final NodeConnectionState... states) {
final Set<NodeConnectionState> statesOfInterest = new HashSet<>();
if (states.length == 0) {
for (final NodeConnectionState state : NodeConnectionState.values()) {
statesOfInterest.add(state);
}
} else {
for (final NodeConnectionState state : states) {
statesOfInterest.add(state);
}
}
return nodeStatuses.entrySet().stream()
.filter(entry -> statesOfInterest.contains(entry.getValue().getState()))
.map(entry -> entry.getKey())
.collect(Collectors.toSet());
}
@Override
public NodeIdentifier getPrimaryNode() {
return nodeStatuses.values().stream()
.filter(status -> status.getRoles().contains(ClusterRoles.PRIMARY_NODE))
.findFirst()
.map(status -> status.getNodeIdentifier())
.orElse(null);
}
@Override
public List<NodeEvent> getNodeEvents(final NodeIdentifier nodeId) {
final CircularFifoQueue<NodeEvent> eventQueue = nodeEvents.get(nodeId);
if (eventQueue == null) {
return Collections.emptyList();
}
synchronized (eventQueue) {
return new ArrayList<>(eventQueue);
}
}
@Override
public void setFlowService(final FlowService flowService) {
if (this.flowService != null) {
throw new IllegalStateException("Flow Service has already been set");
}
this.flowService = flowService;
}
private void addNodeEvent(final NodeIdentifier nodeId, final String event) {
addNodeEvent(nodeId, Severity.INFO, event);
}
private void addNodeEvent(final NodeIdentifier nodeId, final Severity severity, final String message) {
final NodeEvent event = new Event(nodeId.toString(), message, severity);
final CircularFifoQueue<NodeEvent> eventQueue = nodeEvents.computeIfAbsent(nodeId, id -> new CircularFifoQueue<>());
synchronized (eventQueue) {
eventQueue.add(event);
}
}
/**
* Updates the status of the node with the given ID to the given status and returns <code>true</code>
* if successful, <code>false</code> if no node exists with the given ID
*
* @param status the new status of the node
* @return <code>true</code> if the node exists and is updated, <code>false</code> if the node does not exist
*/
// visible for testing.
boolean updateNodeStatus(final NodeConnectionStatus status) {
final NodeIdentifier nodeId = status.getNodeIdentifier();
// In this case, we are using nodeStatuses.put() instead of getting the current value and
// comparing that to the new value and using the one with the largest update id. This is because
// this method is called when something occurs that causes this node to change the status of the
// node in question. We only use comparisons against the current value when we receive an update
// about a node status from a different node, since those may be received out-of-order.
final NodeConnectionStatus currentStatus = nodeStatuses.put(nodeId, status);
final NodeConnectionState currentState = currentStatus == null ? null : currentStatus.getState();
logger.info("Status of {} changed from {} to {}", nodeId, currentStatus, status);
logger.debug("State of cluster nodes is now {}", nodeStatuses);
if (currentState == null || currentState != status.getState()) {
notifyOthersOfNodeStatusChange(status);
}
return true;
}
private void notifyOthersOfNodeStatusChange(final NodeConnectionStatus updatedStatus) {
final Set<NodeIdentifier> nodesToNotify = getNodeIdentifiers(NodeConnectionState.CONNECTED, NodeConnectionState.CONNECTING);
// Do not notify ourselves because we already know about the status update.
nodesToNotify.remove(getLocalNodeIdentifier());
final NodeStatusChangeMessage message = new NodeStatusChangeMessage();
message.setNodeId(updatedStatus.getNodeIdentifier());
message.setNodeConnectionStatus(updatedStatus);
senderListener.notifyNodeStatusChange(nodesToNotify, message);
}
private void disconnectAsynchronously(final DisconnectMessage request, final int attempts, final int retrySeconds) {
final Thread disconnectThread = new Thread(new Runnable() {
@Override
public void run() {
final NodeIdentifier nodeId = request.getNodeId();
for (int i = 0; i < attempts; i++) {
try {
senderListener.disconnect(request);
reportEvent(nodeId, Severity.INFO, "Node disconnected due to " + request.getExplanation());
return;
} catch (final Exception e) {
logger.error("Failed to notify {} that it has been disconnected from the cluster due to {}", request.getNodeId(), request.getExplanation());
try {
Thread.sleep(retrySeconds * 1000L);
} catch (final InterruptedException ie) {
Thread.currentThread().interrupt();
return;
}
}
}
}
}, "Disconnect " + request.getNodeId());
disconnectThread.start();
}
private void requestReconnectionAsynchronously(final ReconnectionRequestMessage request, final int reconnectionAttempts, final int retrySeconds) {
final Thread reconnectionThread = new Thread(new Runnable() {
@Override
public void run() {
// create the request
while (flowService == null) {
try {
Thread.sleep(100L);
} catch (final InterruptedException ie) {
logger.info("Could not send Reconnection request to {} because thread was "
+ "interrupted before FlowService was made available", request.getNodeId());
Thread.currentThread().interrupt();
return;
}
}
for (int i = 0; i < reconnectionAttempts; i++) {
try {
if (NodeConnectionState.CONNECTING != getConnectionState(request.getNodeId())) {
// the node status has changed. It's no longer appropriate to attempt reconnection.
return;
}
request.setDataFlow(new StandardDataFlow(flowService.createDataFlow()));
request.setNodeConnectionStatuses(new ArrayList<>(nodeStatuses.values()));
request.setComponentRevisions(revisionManager.getAllRevisions().stream().map(rev -> ComponentRevision.fromRevision(rev)).collect(Collectors.toList()));
// Issue a reconnection request to the node.
senderListener.requestReconnection(request);
// successfully told node to reconnect -- we're done!
logger.info("Successfully requested that {} join the cluster", request.getNodeId());
return;
} catch (final Exception e) {
logger.warn("Problem encountered issuing reconnection request to node " + request.getNodeId(), e);
eventReporter.reportEvent(Severity.WARNING, EVENT_CATEGORY, "Problem encountered issuing reconnection request to node "
+ request.getNodeId() + " due to: " + e);
}
try {
Thread.sleep(1000L * retrySeconds);
} catch (final InterruptedException ie) {
break;
}
}
// We failed to reconnect too many times. We must now mark node as disconnected.
if (NodeConnectionState.CONNECTING == getConnectionState(request.getNodeId())) {
requestNodeDisconnect(request.getNodeId(), DisconnectionCode.UNABLE_TO_COMMUNICATE,
"Attempted to request that node reconnect to cluster but could not communicate with node");
}
}
}, "Reconnect " + request.getNodeId());
reconnectionThread.start();
}
@Override
public ProtocolMessage handle(final ProtocolMessage protocolMessage) throws ProtocolException {
switch (protocolMessage.getType()) {
case CONNECTION_REQUEST:
return handleConnectionRequest((ConnectionRequestMessage) protocolMessage);
case NODE_STATUS_CHANGE:
handleNodeStatusChange((NodeStatusChangeMessage) protocolMessage);
return null;
default:
throw new ProtocolException("Cannot handle Protocol Message " + protocolMessage + " because it is not of the correct type");
}
}
private void handleNodeStatusChange(final NodeStatusChangeMessage statusChangeMessage) {
final NodeConnectionStatus updatedStatus = statusChangeMessage.getNodeConnectionStatus();
final NodeIdentifier nodeId = statusChangeMessage.getNodeId();
logger.debug("Handling request {}", statusChangeMessage);
boolean updated = false;
while (!updated) {
final NodeConnectionStatus oldStatus = nodeStatuses.get(statusChangeMessage.getNodeId());
if (oldStatus == null || updatedStatus.getUpdateIdentifier() >= oldStatus.getUpdateIdentifier()) {
// Either remove the value from the map or update the map depending on the connection state
if (statusChangeMessage.getNodeConnectionStatus().getState() == NodeConnectionState.REMOVED) {
updated = nodeStatuses.remove(nodeId, oldStatus);
} else {
updated = replaceNodeStatus(nodeId, oldStatus, updatedStatus);
}
if (updated) {
logger.info("Status of {} changed from {} to {}", statusChangeMessage.getNodeId(), oldStatus, updatedStatus);
final NodeConnectionStatus status = statusChangeMessage.getNodeConnectionStatus();
final StringBuilder sb = new StringBuilder();
sb.append("Connection Status changed to ").append(status.getState().toString());
if (status.getDisconnectReason() != null) {
sb.append(" due to ").append(status.getDisconnectReason());
} else if (status.getDisconnectCode() != null) {
sb.append(" due to ").append(status.getDisconnectCode().toString());
}
addNodeEvent(nodeId, sb.toString());
// Update our counter so that we are in-sync with the cluster on the
// most up-to-date version of the NodeConnectionStatus' Update Identifier.
// We do this so that we can accurately compare status updates that are generated
// locally against those generated from other nodes in the cluster.
NodeConnectionStatus.updateIdGenerator(updatedStatus.getUpdateIdentifier());
}
} else {
updated = true;
logger.info("Received Node Status update that indicates that {} should change to {} but disregarding because the current state of {} is newer",
nodeId, updatedStatus, oldStatus);
}
}
}
private NodeIdentifier resolveNodeId(final NodeIdentifier proposedIdentifier) {
final NodeConnectionStatus proposedConnectionStatus = new NodeConnectionStatus(proposedIdentifier, DisconnectionCode.NOT_YET_CONNECTED);
final NodeConnectionStatus existingStatus = nodeStatuses.putIfAbsent(proposedIdentifier, proposedConnectionStatus);
NodeIdentifier resolvedNodeId = proposedIdentifier;
if (existingStatus == null) {
// there is no node with that ID
resolvedNodeId = proposedIdentifier;
logger.debug("No existing node with ID {}; resolved node ID is as-proposed", proposedIdentifier.getId());
} else if (existingStatus.getNodeIdentifier().logicallyEquals(proposedIdentifier)) {
// there is a node with that ID but it's the same node.
resolvedNodeId = proposedIdentifier;
logger.debug("No existing node with ID {}; resolved node ID is as-proposed", proposedIdentifier.getId());
} else {
// there is a node with that ID and it's a different node
resolvedNodeId = new NodeIdentifier(UUID.randomUUID().toString(), proposedIdentifier.getApiAddress(), proposedIdentifier.getApiPort(),
proposedIdentifier.getSocketAddress(), proposedIdentifier.getSocketPort(),
proposedIdentifier.getSiteToSiteAddress(), proposedIdentifier.getSiteToSitePort(), proposedIdentifier.isSiteToSiteSecure());
logger.debug("A node already exists with ID {}. Proposed Node Identifier was {}; existing Node Identifier is {}; Resolved Node Identifier is {}",
proposedIdentifier.getId(), proposedIdentifier, getNodeIdentifier(proposedIdentifier.getId()), resolvedNodeId);
}
return resolvedNodeId;
}
private ConnectionResponseMessage handleConnectionRequest(final ConnectionRequestMessage requestMessage) {
final NodeIdentifier proposedIdentifier = requestMessage.getConnectionRequest().getProposedNodeIdentifier();
final ConnectionRequest requestWithDn = new ConnectionRequest(addRequestorDn(proposedIdentifier, requestMessage.getRequestorDN()));
// Resolve Node identifier.
final NodeIdentifier resolvedNodeId = resolveNodeId(proposedIdentifier);
final ConnectionResponse response = createConnectionResponse(requestWithDn, resolvedNodeId);
final ConnectionResponseMessage responseMessage = new ConnectionResponseMessage();
responseMessage.setConnectionResponse(response);
return responseMessage;
}
private ConnectionResponse createConnectionResponse(final ConnectionRequest request, final NodeIdentifier resolvedNodeIdentifier) {
if (isBlockedByFirewall(resolvedNodeIdentifier.getSocketAddress())) {
// if the socket address is not listed in the firewall, then return a null response
logger.info("Firewall blocked connection request from node " + resolvedNodeIdentifier);
return ConnectionResponse.createBlockedByFirewallResponse();
}
// Set node's status to 'CONNECTING'
NodeConnectionStatus status = getConnectionStatus(resolvedNodeIdentifier);
if (status == null) {
addNodeEvent(resolvedNodeIdentifier, "Connection requested from new node. Setting status to connecting.");
} else {
addNodeEvent(resolvedNodeIdentifier, "Connection requested from existing node. Setting status to connecting");
}
status = new NodeConnectionStatus(resolvedNodeIdentifier, NodeConnectionState.CONNECTING, System.currentTimeMillis());
updateNodeStatus(status);
DataFlow dataFlow = null;
if (flowService != null) {
try {
dataFlow = flowService.createDataFlow();
} catch (final IOException ioe) {
logger.error("Unable to obtain current dataflow from FlowService in order to provide the flow to "
+ resolvedNodeIdentifier + ". Will tell node to try again later", ioe);
}
}
if (dataFlow == null) {
// Create try-later response based on flow retrieval delay to give
// the flow management service a chance to retrieve a current flow
final int tryAgainSeconds = 5;
addNodeEvent(resolvedNodeIdentifier, Severity.WARNING, "Connection requested from node, but manager was unable to obtain current flow. "
+ "Instructing node to try again in " + tryAgainSeconds + " seconds.");
// return try later response
return new ConnectionResponse(tryAgainSeconds);
}
// TODO: Remove the 'null' values here from the ConnectionResponse all together. These
// will no longer be needed for site-to-site once the NCM is gone.
return new ConnectionResponse(resolvedNodeIdentifier, dataFlow, null, null, instanceId,
new ArrayList<>(nodeStatuses.values()),
revisionManager.getAllRevisions().stream().map(rev -> ComponentRevision.fromRevision(rev)).collect(Collectors.toList()));
}
private NodeIdentifier addRequestorDn(final NodeIdentifier nodeId, final String dn) {
return new NodeIdentifier(nodeId.getId(), nodeId.getApiAddress(), nodeId.getApiPort(),
nodeId.getSocketAddress(), nodeId.getSocketPort(),
nodeId.getSiteToSiteAddress(), nodeId.getSiteToSitePort(), nodeId.isSiteToSiteSecure(), dn);
}
@Override
public boolean canHandle(final ProtocolMessage msg) {
return MessageType.CONNECTION_REQUEST == msg.getType() || MessageType.NODE_STATUS_CHANGE == msg.getType();
}
private boolean isMutableRequest(final String method) {
return "DELETE".equalsIgnoreCase(method) || "POST".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method);
}
/**
* Callback that is called after an HTTP Request has been replicated to nodes in the cluster.
* This allows us to disconnect nodes that did not complete the request, if applicable.
*/
@Override
public void afterRequest(final String uriPath, final String method, final Set<NodeResponse> nodeResponses) {
final boolean mutableRequest = isMutableRequest(method);
/*
* Nodes that encountered issues handling the request are marked as
* disconnected for mutable requests (e.g., post, put, delete). For
* other requests (e.g., get, head), the nodes remain in their current
* state even if they had problems handling the request.
*/
if (mutableRequest) {
final HttpResponseMerger responseMerger = new StandardHttpResponseMerger();
final Set<NodeResponse> problematicNodeResponses = responseMerger.getProblematicNodeResponses(nodeResponses);
// all nodes failed
final boolean allNodesFailed = problematicNodeResponses.size() == nodeResponses.size();
// some nodes had a problematic response because of a missing counter, ensure the are not disconnected
final boolean someNodesFailedMissingCounter = !problematicNodeResponses.isEmpty()
&& problematicNodeResponses.size() < nodeResponses.size() && isMissingCounter(problematicNodeResponses, uriPath);
// ensure nodes stay connected in certain scenarios
if (allNodesFailed) {
logger.warn("All nodes failed to process URI {} {}. As a result, no node will be disconnected from cluster", method, uriPath);
return;
}
if (someNodesFailedMissingCounter) {
return;
}
// disconnect problematic nodes
if (!problematicNodeResponses.isEmpty() && problematicNodeResponses.size() < nodeResponses.size()) {
logger.warn(String.format("The following nodes failed to process URI %s '%s'. Requesting each node to disconnect from cluster: ", uriPath, problematicNodeResponses));
for (final NodeResponse nodeResponse : problematicNodeResponses) {
requestNodeDisconnect(nodeResponse.getNodeId(), DisconnectionCode.FAILED_TO_SERVICE_REQUEST, "Failed to process URI to " + method + " " + uriPath);
}
}
}
}
/**
* Determines if all problematic responses were due to 404 NOT_FOUND. Assumes that problematicNodeResponses is not empty and is not comprised of responses from all nodes in the cluster (at least
* one node contained the counter in question).
*
* @param problematicNodeResponses The problematic node responses
* @param uriPath The path of the URI for the request
* @return Whether all problematic node responses were due to a missing counter
*/
private boolean isMissingCounter(final Set<NodeResponse> problematicNodeResponses, final String uriPath) {
if (COUNTER_URI_PATTERN.matcher(uriPath).matches()) {
boolean notFound = true;
for (final NodeResponse problematicResponse : problematicNodeResponses) {
if (problematicResponse.getStatus() != 404) {
notFound = false;
break;
}
}
return notFound;
}
return false;
}
@Override
public void setConnected(final boolean connected) {
this.connected = connected;
}
@Override
public boolean isConnected() {
return connected;
}
}

View File

@ -17,23 +17,25 @@
package org.apache.nifi.cluster.coordination.node;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
public class ClusterNode {
private final NodeIdentifier nodeId;
private NodeConnectionStatus connectionStatus = new NodeConnectionStatus(NodeConnectionState.DISCONNECTED, DisconnectionCode.NOT_YET_CONNECTED);
public final class NodeStateManager {
private final ConcurrentMap<NodeIdentifier, NodeConnectionStatus> nodeStatuses = new ConcurrentHashMap<>();
public ClusterNode(final NodeIdentifier nodeId) {
this.nodeId = nodeId;
public NodeConnectionStatus getConnectionStatus(final NodeIdentifier nodeId) {
return nodeStatuses.get(nodeId);
}
public NodeIdentifier getIdentifier() {
return nodeId;
public void setConnectionStatus(final NodeIdentifier nodeId, final NodeConnectionStatus status) {
nodeStatuses.put(nodeId, status);
// TODO: Notify other nodes
}
public NodeConnectionStatus getConnectionStatus() {
return connectionStatus;
public boolean transitionConnectionStatus(final NodeIdentifier nodeId, final NodeConnectionStatus currentStatus, final NodeConnectionStatus newStatus) {
return nodeStatuses.replace(nodeId, currentStatus, newStatus);
// TODO: Notify other nodes
}
}
}

View File

@ -19,6 +19,7 @@ package org.apache.nifi.cluster.event;
import java.util.Date;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.reporting.Severity;
/**
* Events describe the occurrence of something noteworthy. They record the event's source, a timestamp, a description, and a category.
@ -26,21 +27,11 @@ import org.apache.commons.lang3.StringUtils;
*
* @Immutable
*/
public class Event {
public static enum Category {
DEBUG,
INFO,
WARN
}
public class Event implements NodeEvent {
private final String source;
private final long timestamp;
private final Category category;
private final Severity severity;
private final String message;
/**
@ -50,7 +41,7 @@ public class Event {
* @param message the description
*/
public Event(final String source, final String message) {
this(source, message, Category.INFO);
this(source, message, Severity.INFO);
}
/**
@ -58,10 +49,10 @@ public class Event {
*
* @param source the source
* @param message the description
* @param category the event category
* @param severity the event severity
*/
public Event(final String source, final String message, final Category category) {
this(source, message, category, new Date().getTime());
public Event(final String source, final String message, final Severity severity) {
this(source, message, severity, new Date().getTime());
}
/**
@ -72,7 +63,7 @@ public class Event {
* @param timestamp the time of occurrence
*/
public Event(final String source, final String message, final long timestamp) {
this(source, message, Category.INFO, timestamp);
this(source, message, Severity.INFO, timestamp);
}
/**
@ -80,16 +71,15 @@ public class Event {
*
* @param source the source
* @param message the description
* @param category the event category
* @param severity the event category
* @param timestamp the time of occurrence
*/
public Event(final String source, final String message, final Category category, final long timestamp) {
public Event(final String source, final String message, final Severity severity, final long timestamp) {
if (StringUtils.isBlank(source)) {
throw new IllegalArgumentException("Source may not be empty or null.");
} else if (StringUtils.isBlank(message)) {
throw new IllegalArgumentException("Event message may not be empty or null.");
} else if (category == null) {
} else if (severity == null) {
throw new IllegalArgumentException("Event category may not be null.");
} else if (timestamp < 0) {
throw new IllegalArgumentException("Timestamp may not be negative: " + timestamp);
@ -97,24 +87,27 @@ public class Event {
this.source = source;
this.message = message;
this.category = category;
this.severity = severity;
this.timestamp = timestamp;
}
public Category getCategory() {
return category;
@Override
public Severity getSeverity() {
return severity;
}
@Override
public String getMessage() {
return message;
}
@Override
public String getSource() {
return source;
}
@Override
public long getTimestamp() {
return timestamp;
}
}

View File

@ -1,62 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.event;
import java.util.List;
/**
* Manages an ordered list of events. The event history size dictates the total number of events to manage for a given source at a given time. When the size is exceeded, the oldest event for that
* source is evicted.
*
*/
public interface EventManager {
/**
* Adds an event to the manager.
*
* @param event an Event
*/
void addEvent(Event event);
/**
* Returns a list of events for a given source sorted by the event's timestamp where the most recent event is first in the list.
*
* @param eventSource the source
*
* @return the list of events
*/
List<Event> getEvents(String eventSource);
/*
* Returns the most recent event for the source. If no events exist, then
* null is returned.
*/
Event getMostRecentEvent(String eventSource);
/*
* Clears all events for the given source.
*/
void clearEventHistory(String eventSource);
/**
* Returns the history size.
*
* @return the history size
*/
int getEventHistorySize();
}

View File

@ -1,141 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.event.impl;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.PriorityQueue;
import java.util.Queue;
import org.apache.nifi.cluster.event.Event;
import org.apache.nifi.cluster.event.EventManager;
/**
* Implements the EventManager.
*
*/
public class EventManagerImpl implements EventManager {
/**
* associates the source ID with an ordered queue of events, ordered by most recent event
*/
private final Map<String, Queue<Event>> eventsMap = new HashMap<>();
/**
* the number of events to maintain for a given source
*/
private final int eventHistorySize;
/**
* Creates an instance.
*
* @param eventHistorySize the number of events to manage for a given source. Value must be positive.
*/
public EventManagerImpl(final int eventHistorySize) {
if (eventHistorySize <= 0) {
throw new IllegalArgumentException("Event history size must be positive: " + eventHistorySize);
}
this.eventHistorySize = eventHistorySize;
}
@Override
public void addEvent(final Event event) {
if (event == null) {
throw new IllegalArgumentException("Event may not be null.");
}
Queue<Event> events = eventsMap.get(event.getSource());
if (events == null) {
// no events from this source, so add a new queue to the map
events = new PriorityQueue<>(eventHistorySize, createEventComparator());
eventsMap.put(event.getSource(), events);
}
// add event
events.add(event);
// if we exceeded the history size, then evict the oldest event
if (events.size() > eventHistorySize) {
removeOldestEvent(events);
}
}
@Override
public List<Event> getEvents(final String eventSource) {
final Queue<Event> events = eventsMap.get(eventSource);
if (events == null) {
return Collections.EMPTY_LIST;
} else {
return Collections.unmodifiableList(new ArrayList<>(events));
}
}
@Override
public int getEventHistorySize() {
return eventHistorySize;
}
@Override
public Event getMostRecentEvent(final String eventSource) {
final Queue<Event> events = eventsMap.get(eventSource);
if (events == null) {
return null;
} else {
return events.peek();
}
}
@Override
public void clearEventHistory(final String eventSource) {
eventsMap.remove(eventSource);
}
private Comparator createEventComparator() {
return new Comparator<Event>() {
@Override
public int compare(final Event o1, final Event o2) {
// orders events by most recent first
return (int) (o2.getTimestamp() - o1.getTimestamp());
}
};
}
private void removeOldestEvent(final Collection<Event> events) {
if (events.isEmpty()) {
return;
}
Event oldestEvent = null;
for (final Event event : events) {
if (oldestEvent == null || oldestEvent.getTimestamp() > event.getTimestamp()) {
oldestEvent = event;
}
}
events.remove(oldestEvent);
}
}

View File

@ -1,55 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.flow;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.cluster.protocol.StandardDataFlow;
/**
* A dataflow with additional information about the cluster.
*
*/
public class ClusterDataFlow {
private final StandardDataFlow dataFlow;
private final NodeIdentifier primaryNodeId;
private final byte[] controllerServices;
private final byte[] reportingTasks;
public ClusterDataFlow(final StandardDataFlow dataFlow, final NodeIdentifier primaryNodeId, final byte[] controllerServices, final byte[] reportingTasks) {
this.dataFlow = dataFlow;
this.primaryNodeId = primaryNodeId;
this.controllerServices = controllerServices;
this.reportingTasks = reportingTasks;
}
public byte[] getControllerServices() {
return controllerServices;
}
public byte[] getReportingTasks() {
return reportingTasks;
}
public NodeIdentifier getPrimaryNodeId() {
return primaryNodeId;
}
public StandardDataFlow getDataFlow() {
return dataFlow;
}
}

View File

@ -1,60 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.flow;
/**
* A data access object for loading and saving the flow managed by the cluster.
*
*/
public interface DataFlowDao {
/**
* Loads the cluster's dataflow.
*
* @return the dataflow or null if no dataflow exists
*
* @throws DaoException if the dataflow was unable to be loaded
*/
ClusterDataFlow loadDataFlow() throws DaoException;
/**
* Saves the cluster's dataflow.
*
*
* @param dataFlow flow
* @throws DaoException if the dataflow was unable to be saved
*/
void saveDataFlow(ClusterDataFlow dataFlow) throws DaoException;
/**
* Sets the state of the dataflow. If the dataflow does not exist, then an exception is thrown.
*
* @param flowState the state of the dataflow
*
* @throws DaoException if the state was unable to be updated
*/
void setPersistedFlowState(PersistedFlowState flowState) throws DaoException;
/**
* Gets the state of the dataflow.
*
* @return the state of the dataflow
*
* @throws DaoException if the state was unable to be retrieved
*/
PersistedFlowState getPersistedFlowState() throws DaoException;
}

View File

@ -1,124 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.flow;
import java.util.Set;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
/**
* A service for managing the cluster's flow. The service will attempt to keep the cluster's dataflow current while respecting the value of the configured retrieval delay.
*
* The eligible retrieval time is reset with the configured delay every time the flow state is set to STALE. If the state is set to UNKNOWN or CURRENT, then the flow will not be retrieved.
*
* Clients must call start() and stop() to initialize and stop the instance.
*
*/
public interface DataFlowManagementService {
/**
* Starts the instance. Start may only be called if the instance is not running.
*/
void start();
/**
* Stops the instance. Stop may only be called if the instance is running.
*/
void stop();
/**
* @return true if the instance is started; false otherwise.
*/
boolean isRunning();
/**
* Loads the dataflow.
*
* @return the dataflow or null if no dataflow exists
*/
ClusterDataFlow loadDataFlow();
/**
* Updates the dataflow with the given primary node identifier.
*
* @param nodeId the node identifier
*
* @throws DaoException if the update failed
*/
void updatePrimaryNode(NodeIdentifier nodeId) throws DaoException;
/**
* Updates the dataflow with the given serialized form of the Controller Services that are to exist on the NCM.
*
* @param serializedControllerServices services
* @throws DaoException ex
*/
void updateControllerServices(byte[] serializedControllerServices) throws DaoException;
/**
* Updates the dataflow with the given serialized form of Reporting Tasks that are to exist on the NCM.
*
* @param serializedReportingTasks tasks
* @throws DaoException ex
*/
void updateReportingTasks(byte[] serializedReportingTasks) throws DaoException;
/**
* Sets the state of the flow.
*
* @param flowState the state
*
* @see PersistedFlowState
*/
void setPersistedFlowState(PersistedFlowState flowState);
/**
* @return the state of the flow
*/
PersistedFlowState getPersistedFlowState();
/**
* @return true if the flow is current; false otherwise.
*/
boolean isFlowCurrent();
/**
* Sets the node identifiers to use when attempting to retrieve the flow.
*
* @param nodeIds the node identifiers
*/
void setNodeIds(Set<NodeIdentifier> nodeIds);
/**
* Returns the set of node identifiers the service is using to retrieve the flow.
*
* @return the set of node identifiers the service is using to retrieve the flow.
*/
Set<NodeIdentifier> getNodeIds();
/**
* @return the retrieval delay in seconds
*/
int getRetrievalDelaySeconds();
/**
* Sets the retrieval delay.
*
* @param delay the retrieval delay in seconds
*/
void setRetrievalDelay(String delay);
}

View File

@ -1,40 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.flow;
/**
* Represents the exceptional case when a caller is requesting the current flow, but a current flow is not available.
*
*/
public class StaleFlowException extends RuntimeException {
public StaleFlowException(String message, Throwable cause) {
super(message, cause);
}
public StaleFlowException(String message) {
super(message);
}
public StaleFlowException(Throwable cause) {
super(cause);
}
public StaleFlowException() {
}
}

View File

@ -1,595 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.flow.impl;
import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.FilenameFilter;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.security.MessageDigest;
import java.security.NoSuchAlgorithmException;
import java.util.Arrays;
import java.util.UUID;
import javax.xml.bind.JAXBContext;
import javax.xml.bind.JAXBException;
import javax.xml.bind.Marshaller;
import javax.xml.bind.Unmarshaller;
import javax.xml.bind.annotation.XmlRootElement;
import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
import javax.xml.parsers.DocumentBuilder;
import javax.xml.parsers.DocumentBuilderFactory;
import javax.xml.transform.OutputKeys;
import javax.xml.transform.Transformer;
import javax.xml.transform.TransformerFactory;
import javax.xml.transform.dom.DOMSource;
import javax.xml.transform.stream.StreamResult;
import org.apache.commons.compress.archivers.ArchiveEntry;
import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
import org.apache.commons.compress.archivers.tar.TarArchiveInputStream;
import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream;
import org.apache.nifi.cluster.flow.ClusterDataFlow;
import org.apache.nifi.cluster.flow.DaoException;
import org.apache.nifi.cluster.flow.DataFlowDao;
import org.apache.nifi.cluster.flow.PersistedFlowState;
import org.apache.nifi.cluster.protocol.DataFlow;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.cluster.protocol.StandardDataFlow;
import org.apache.nifi.cluster.protocol.jaxb.message.NodeIdentifierAdapter;
import org.apache.nifi.logging.NiFiLog;
import org.apache.nifi.stream.io.BufferedInputStream;
import org.apache.nifi.stream.io.BufferedOutputStream;
import org.apache.nifi.stream.io.ByteArrayInputStream;
import org.apache.nifi.stream.io.StreamUtils;
import org.apache.nifi.util.file.FileUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.w3c.dom.Document;
import org.w3c.dom.Element;
/**
* Implements the FlowDao interface. The implementation tracks the state of the dataflow by annotating the filename of the flow state file. Specifically, the implementation correlates
* PersistedFlowState states to filename extensions. The correlation is as follows:
* <ul>
* <li> CURRENT maps to flow.xml </li>
* <li> STALE maps to flow.xml.stale </li>
* <li> UNKNOWN maps to flow.xml.unknown </li>
* </ul>
* Whenever the flow state changes, the flow state file's name is updated to denote its state.
*
* The implementation also provides for a restore directory that may be configured for higher availability. At instance creation, if the primary or restore directories have multiple flow state files,
* an exception is thrown. If the primary directory has a current flow state file, but the restore directory does not, then the primary flow state file is copied to the restore directory. If the
* restore directory has a current flow state file, but the primary directory does not, then the restore flow state file is copied to the primary directory. If both the primary and restore directories
* have a current flow state file and the files are different, then an exception is thrown.
*
* When the flow state file is saved, it is always saved first to the restore directory followed by a save to the primary directory. When the flow state file is loaded, a check is made to verify that
* the primary and restore flow state files are both current. If either is not current, then an exception is thrown. The primary flow state file is always read when the load method is called.
*
*/
public class DataFlowDaoImpl implements DataFlowDao {
private final File primaryDirectory;
private final File restoreDirectory;
private final boolean autoStart;
private final String generatedRootGroupId = UUID.randomUUID().toString();
public static final String STALE_EXT = ".stale";
public static final String UNKNOWN_EXT = ".unknown";
public static final String FLOW_PACKAGE = "flow.tar";
public static final String FLOW_XML_FILENAME = "flow.xml";
public static final String TEMPLATES_FILENAME = "templates.xml";
public static final String SNIPPETS_FILENAME = "snippets.xml";
public static final String CONTROLLER_SERVICES_FILENAME = "controller-services.xml";
public static final String REPORTING_TASKS_FILENAME = "reporting-tasks.xml";
public static final String CLUSTER_INFO_FILENAME = "cluster-info.xml";
private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(DataFlowDaoImpl.class));
public DataFlowDaoImpl(final File primaryDirectory) throws DaoException {
this(primaryDirectory, null, false);
}
public DataFlowDaoImpl(final File primaryDirectory, final File restoreDirectory, final boolean autoStart) throws DaoException {
// sanity check that primary directory is a directory, creating it if necessary
if (primaryDirectory == null) {
throw new IllegalArgumentException("Primary directory may not be null.");
} else if (!primaryDirectory.exists()) {
if (!primaryDirectory.mkdir()) {
throw new DaoException(String.format("Failed to create primary directory '%s'", primaryDirectory.getAbsolutePath()));
}
} else if (!primaryDirectory.isDirectory()) {
throw new IllegalArgumentException("Primary directory must be a directory.");
}
this.autoStart = autoStart;
try {
this.primaryDirectory = primaryDirectory;
this.restoreDirectory = restoreDirectory;
if (restoreDirectory == null) {
// check that we have exactly one current flow state file
ensureSingleCurrentStateFile(primaryDirectory);
} else {
// check that restore directory is a directory, creating it if necessary
FileUtils.ensureDirectoryExistAndCanAccess(restoreDirectory);
// check that restore directory is not the same as the primary directory
if (primaryDirectory.getAbsolutePath().equals(restoreDirectory.getAbsolutePath())) {
throw new IllegalArgumentException(String.format("Primary directory '%s' is the same as restore directory '%s' ",
primaryDirectory.getAbsolutePath(), restoreDirectory.getAbsolutePath()));
}
final File[] primaryFlowStateFiles = getFlowStateFiles(primaryDirectory);
final File[] restoreFlowStateFiles = getFlowStateFiles(restoreDirectory);
// if more than one state file in either primary or restore, then throw exception
if (primaryFlowStateFiles.length > 1) {
throw new IllegalStateException(String.format("Found multiple dataflow state files in primary directory '%s'", primaryDirectory));
} else if (restoreFlowStateFiles.length > 1) {
throw new IllegalStateException(String.format("Found multiple dataflow state files in restore directory '%s'", restoreDirectory));
}
// check that the single primary state file we found is current or create a new one
final File primaryFlowStateFile = ensureSingleCurrentStateFile(primaryDirectory);
// check that the single restore state file we found is current or create a new one
final File restoreFlowStateFile = ensureSingleCurrentStateFile(restoreDirectory);
// if there was a difference in flow state file directories, then copy the appropriate files
if (restoreFlowStateFiles.length == 0 && primaryFlowStateFiles.length != 0) {
// copy primary state file to restore
FileUtils.copyFile(primaryFlowStateFile, restoreFlowStateFile, false, false, logger);
} else if (primaryFlowStateFiles.length == 0 && restoreFlowStateFiles.length != 0) {
// copy restore state file to primary
FileUtils.copyFile(restoreFlowStateFile, primaryFlowStateFile, false, false, logger);
} else {
// sync the primary copy with the restore copy
syncWithRestore(primaryFlowStateFile, restoreFlowStateFile);
}
}
} catch (final IOException | IllegalArgumentException | IllegalStateException | JAXBException ex) {
throw new DaoException(ex);
}
}
private void syncWithRestore(final File primaryFile, final File restoreFile) throws IOException {
try (final FileInputStream primaryFis = new FileInputStream(primaryFile);
final TarArchiveInputStream primaryIn = new TarArchiveInputStream(primaryFis);
final FileInputStream restoreFis = new FileInputStream(restoreFile);
final TarArchiveInputStream restoreIn = new TarArchiveInputStream(restoreFis)) {
final ArchiveEntry primaryEntry = primaryIn.getNextEntry();
final ArchiveEntry restoreEntry = restoreIn.getNextEntry();
if (primaryEntry == null && restoreEntry == null) {
return;
}
if (primaryEntry == null && restoreEntry != null || primaryEntry != null && restoreEntry == null) {
throw new IllegalStateException(String.format("Primary file '%s' is different than restore file '%s'",
primaryFile.getAbsoluteFile(), restoreFile.getAbsolutePath()));
}
final byte[] primaryMd5 = calculateMd5(primaryIn);
final byte[] restoreMd5 = calculateMd5(restoreIn);
if (!Arrays.equals(primaryMd5, restoreMd5)) {
throw new IllegalStateException(String.format("Primary file '%s' is different than restore file '%s'",
primaryFile.getAbsoluteFile(), restoreFile.getAbsolutePath()));
}
}
}
private byte[] calculateMd5(final InputStream in) throws IOException {
final MessageDigest digest;
try {
digest = MessageDigest.getInstance("MD5");
} catch (final NoSuchAlgorithmException nsae) {
throw new IOException(nsae);
}
int len;
final byte[] buffer = new byte[8192];
while ((len = in.read(buffer)) > -1) {
if (len > 0) {
digest.update(buffer, 0, len);
}
}
return digest.digest();
}
@Override
public ClusterDataFlow loadDataFlow() throws DaoException {
try {
return parseDataFlow(getExistingFlowStateFile(primaryDirectory));
} catch (final IOException | JAXBException ex) {
throw new DaoException(ex);
}
}
@Override
public void saveDataFlow(final ClusterDataFlow dataFlow) throws DaoException {
try {
final File primaryStateFile = getFlowStateFile(primaryDirectory);
// write to restore before writing to primary in case primary experiences problems
if (restoreDirectory != null) {
final File restoreStateFile = getFlowStateFile(restoreDirectory);
if (restoreStateFile == null) {
if (primaryStateFile == null) {
writeDataFlow(createNewFlowStateFile(restoreDirectory), dataFlow);
} else {
throw new DaoException(String.format("Unable to save dataflow because dataflow state file in primary directory "
+ "'%s' exists, but it does not exist in the restore directory '%s'",
primaryDirectory.getAbsolutePath(), restoreDirectory.getAbsolutePath()));
}
} else {
if (primaryStateFile == null) {
throw new DaoException(String.format("Unable to save dataflow because dataflow state file in restore directory "
+ "'%s' exists, but it does not exist in the primary directory '%s'",
restoreDirectory.getAbsolutePath(), primaryDirectory.getAbsolutePath()));
} else {
final PersistedFlowState primaryFlowState = getPersistedFlowState(primaryStateFile);
final PersistedFlowState restoreFlowState = getPersistedFlowState(restoreStateFile);
if (primaryFlowState == restoreFlowState) {
writeDataFlow(restoreStateFile, dataFlow);
} else {
throw new DaoException(String.format("Unable to save dataflow because state file in primary directory "
+ "'%s' has state '%s', but the state file in the restore directory '%s' has state '%s'",
primaryDirectory.getAbsolutePath(), primaryFlowState, restoreDirectory.getAbsolutePath(), restoreFlowState));
}
}
}
}
// write dataflow to primary
if (primaryStateFile == null) {
writeDataFlow(createNewFlowStateFile(primaryDirectory), dataFlow);
} else {
writeDataFlow(primaryStateFile, dataFlow);
}
} catch (final IOException | JAXBException ex) {
throw new DaoException(ex);
}
}
@Override
public PersistedFlowState getPersistedFlowState() {
// trust restore over primary if configured for restore
if (restoreDirectory == null) {
return getPersistedFlowState(getExistingFlowStateFile(primaryDirectory));
} else {
return getPersistedFlowState(getExistingFlowStateFile(restoreDirectory));
}
}
@Override
public void setPersistedFlowState(final PersistedFlowState flowState) throws DaoException {
// rename restore before primary if configured for restore
if (restoreDirectory != null) {
renameFlowStateFile(getExistingFlowStateFile(restoreDirectory), flowState);
}
renameFlowStateFile(getExistingFlowStateFile(primaryDirectory), flowState);
}
private File ensureSingleCurrentStateFile(final File dir) throws IOException, JAXBException {
// ensure that we have at most one state file and if we have one, it is current
final File[] directoryFlowStateFiles = getFlowStateFiles(dir);
if (directoryFlowStateFiles.length > 1) {
throw new DaoException(String.format("Found multiple dataflow state files in directory '%s'", dir));
} else if (directoryFlowStateFiles.length == 0) {
// create a new file if none exist
return createNewFlowStateFile(dir);
} else {
// check that the single flow state file is current
final PersistedFlowState flowState = getPersistedFlowState(directoryFlowStateFiles[0]);
if (PersistedFlowState.CURRENT == flowState) {
return directoryFlowStateFiles[0];
} else {
throw new DaoException(String.format("Dataflow state file '%s' must be current.", directoryFlowStateFiles[0].getAbsolutePath()));
}
}
}
private PersistedFlowState getPersistedFlowState(final File file) {
final String path = file.getAbsolutePath();
if (path.endsWith(STALE_EXT)) {
return PersistedFlowState.STALE;
} else if (path.endsWith(UNKNOWN_EXT)) {
return PersistedFlowState.UNKNOWN;
} else {
return PersistedFlowState.CURRENT;
}
}
private File getFlowStateFile(final File dir) {
final File[] files = getFlowStateFiles(dir);
if (files.length > 1) {
throw new IllegalStateException(String.format("Expected at most one dataflow state file, but found %s files.", files.length));
} else if (files.length == 0) {
return null;
} else {
return files[0];
}
}
private File getExistingFlowStateFile(final File dir) {
final File file = getFlowStateFile(dir);
if (file == null) {
throw new IllegalStateException(String.format("Expected a dataflow state file, but none existed in directory '%s'", dir.getAbsolutePath()));
}
return file;
}
private File[] getFlowStateFiles(final File dir) {
final File[] files = dir.listFiles(new FilenameFilter() {
@Override
public boolean accept(File dir, String name) {
return name.equals(FLOW_PACKAGE) || name.endsWith(STALE_EXT) || name.endsWith(UNKNOWN_EXT);
}
});
if (files == null) {
return new File[0];
} else {
return files;
}
}
private File removeStateFileExtension(final File file) {
final String path = file.getAbsolutePath();
final int stateFileExtIndex;
if (path.endsWith(STALE_EXT)) {
stateFileExtIndex = path.lastIndexOf(STALE_EXT);
} else if (path.endsWith(UNKNOWN_EXT)) {
stateFileExtIndex = path.lastIndexOf(UNKNOWN_EXT);
} else {
stateFileExtIndex = path.length();
}
return new File(path.substring(0, stateFileExtIndex));
}
private File addStateFileExtension(final File file, final PersistedFlowState state) {
switch (state) {
case CURRENT: {
return file;
}
case STALE: {
return new File(file.getAbsolutePath() + STALE_EXT);
}
case UNKNOWN: {
return new File(file.getAbsolutePath() + UNKNOWN_EXT);
}
default: {
throw new RuntimeException("Unsupported PersistedFlowState Enum value: " + state);
}
}
}
private File createNewFlowStateFile(final File dir) throws IOException, JAXBException {
final File stateFile = new File(dir, FLOW_PACKAGE);
stateFile.createNewFile();
writeDataFlow(stateFile, new ClusterDataFlow(null, null, new byte[0], new byte[0]), new ClusterMetadata());
return stateFile;
}
private byte[] getEmptyFlowBytes() throws IOException {
try {
final DocumentBuilderFactory docBuilderFactory = DocumentBuilderFactory.newInstance();
docBuilderFactory.setNamespaceAware(true);
final DocumentBuilder docBuilder = docBuilderFactory.newDocumentBuilder();
final Document document = docBuilder.newDocument();
final Element controller = document.createElement("flowController");
document.appendChild(controller);
controller.appendChild(createTextElement(document, "maxThreadCount", "15"));
final Element rootGroup = document.createElement("rootGroup");
rootGroup.appendChild(createTextElement(document, "id", generatedRootGroupId));
rootGroup.appendChild(createTextElement(document, "name", "NiFi Flow"));
// create the position element
final Element positionElement = createTextElement(document, "position", "");
positionElement.setAttribute("x", "0.0");
positionElement.setAttribute("y", "0.0");
rootGroup.appendChild(positionElement);
rootGroup.appendChild(createTextElement(document, "comment", ""));
controller.appendChild(rootGroup);
final Transformer transformer = TransformerFactory.newInstance().newTransformer();
transformer.setOutputProperty("{http://xml.apache.org/xslt}indent-amount", "2");
transformer.setOutputProperty(OutputKeys.INDENT, "yes");
final DOMSource source = new DOMSource(document);
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
final StreamResult result = new StreamResult(baos);
transformer.transform(source, result);
return baos.toByteArray();
} catch (final Exception e) {
throw new IOException(e);
}
}
private Element createTextElement(final Document document, final String elementName, final String value) {
final Element element = document.createElement(elementName);
element.setTextContent(value);
return element;
}
private void renameFlowStateFile(final File flowStateFile, final PersistedFlowState newState) throws DaoException {
final PersistedFlowState existingState = getPersistedFlowState(flowStateFile);
if (existingState != newState) {
final File newFlowStateFile = addStateFileExtension(removeStateFileExtension(flowStateFile), newState);
if (flowStateFile.renameTo(newFlowStateFile) == false) {
throw new DaoException(
String.format("Failed to rename flow state file '%s' to new name '%s'", flowStateFile.getAbsolutePath(), newFlowStateFile.getAbsolutePath()));
}
}
}
private ClusterDataFlow parseDataFlow(final File file) throws IOException, JAXBException, DaoException {
byte[] flowBytes = new byte[0];
byte[] templateBytes = new byte[0];
byte[] snippetBytes = new byte[0];
byte[] clusterInfoBytes = new byte[0];
byte[] controllerServiceBytes = new byte[0];
byte[] reportingTaskBytes = new byte[0];
try (final InputStream inStream = new FileInputStream(file);
final TarArchiveInputStream tarIn = new TarArchiveInputStream(new BufferedInputStream(inStream))) {
TarArchiveEntry tarEntry;
while ((tarEntry = tarIn.getNextTarEntry()) != null) {
switch (tarEntry.getName()) {
case FLOW_XML_FILENAME:
flowBytes = new byte[(int) tarEntry.getSize()];
StreamUtils.fillBuffer(tarIn, flowBytes, true);
break;
case TEMPLATES_FILENAME:
templateBytes = new byte[(int) tarEntry.getSize()];
StreamUtils.fillBuffer(tarIn, templateBytes, true);
break;
case SNIPPETS_FILENAME:
snippetBytes = new byte[(int) tarEntry.getSize()];
StreamUtils.fillBuffer(tarIn, snippetBytes, true);
break;
case CLUSTER_INFO_FILENAME:
clusterInfoBytes = new byte[(int) tarEntry.getSize()];
StreamUtils.fillBuffer(tarIn, clusterInfoBytes, true);
break;
case CONTROLLER_SERVICES_FILENAME:
controllerServiceBytes = new byte[(int) tarEntry.getSize()];
StreamUtils.fillBuffer(tarIn, controllerServiceBytes, true);
break;
case REPORTING_TASKS_FILENAME:
reportingTaskBytes = new byte[(int) tarEntry.getSize()];
StreamUtils.fillBuffer(tarIn, reportingTaskBytes, true);
break;
default:
throw new DaoException("Found Unexpected file in dataflow configuration: " + tarEntry.getName());
}
}
}
final ClusterMetadata clusterMetadata;
if (clusterInfoBytes.length == 0) {
clusterMetadata = null;
} else {
final Unmarshaller clusterMetadataUnmarshaller = ClusterMetadata.jaxbCtx.createUnmarshaller();
clusterMetadata = (ClusterMetadata) clusterMetadataUnmarshaller.unmarshal(new ByteArrayInputStream(clusterInfoBytes));
}
final StandardDataFlow dataFlow = new StandardDataFlow(flowBytes, snippetBytes);
dataFlow.setAutoStartProcessors(autoStart);
return new ClusterDataFlow(dataFlow, clusterMetadata == null ? null : clusterMetadata.getPrimaryNodeId(), controllerServiceBytes, reportingTaskBytes);
}
private void writeDataFlow(final File file, final ClusterDataFlow clusterDataFlow) throws IOException, JAXBException {
// setup the cluster metadata
final ClusterMetadata clusterMetadata = new ClusterMetadata();
clusterMetadata.setPrimaryNodeId(clusterDataFlow.getPrimaryNodeId());
// write to disk
writeDataFlow(file, clusterDataFlow, clusterMetadata);
}
private void writeTarEntry(final TarArchiveOutputStream tarOut, final String filename, final byte[] bytes) throws IOException {
final TarArchiveEntry flowEntry = new TarArchiveEntry(filename);
flowEntry.setSize(bytes.length);
tarOut.putArchiveEntry(flowEntry);
tarOut.write(bytes);
tarOut.closeArchiveEntry();
}
private void writeDataFlow(final File file, final ClusterDataFlow clusterDataFlow, final ClusterMetadata clusterMetadata) throws IOException, JAXBException {
try (final OutputStream fos = new FileOutputStream(file);
final TarArchiveOutputStream tarOut = new TarArchiveOutputStream(new BufferedOutputStream(fos))) {
final DataFlow dataFlow = clusterDataFlow.getDataFlow();
if (dataFlow == null) {
writeTarEntry(tarOut, FLOW_XML_FILENAME, getEmptyFlowBytes());
writeTarEntry(tarOut, SNIPPETS_FILENAME, new byte[0]);
} else {
writeTarEntry(tarOut, FLOW_XML_FILENAME, dataFlow.getFlow());
writeTarEntry(tarOut, SNIPPETS_FILENAME, dataFlow.getSnippets());
}
writeTarEntry(tarOut, CONTROLLER_SERVICES_FILENAME, clusterDataFlow.getControllerServices());
writeTarEntry(tarOut, REPORTING_TASKS_FILENAME, clusterDataFlow.getReportingTasks());
final ByteArrayOutputStream baos = new ByteArrayOutputStream(256);
writeClusterMetadata(clusterMetadata, baos);
final byte[] clusterInfoBytes = baos.toByteArray();
writeTarEntry(tarOut, CLUSTER_INFO_FILENAME, clusterInfoBytes);
}
}
private void writeClusterMetadata(final ClusterMetadata clusterMetadata, final OutputStream os) throws IOException, JAXBException {
// write cluster metadata to output stream
final Marshaller marshaller = ClusterMetadata.jaxbCtx.createMarshaller();
marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true);
marshaller.setProperty(Marshaller.JAXB_FRAGMENT, true);
marshaller.setProperty(Marshaller.JAXB_ENCODING, "UTF-8");
marshaller.marshal(clusterMetadata, os);
}
@XmlRootElement(name = "clusterMetadata")
private static class ClusterMetadata {
private NodeIdentifier primaryNodeId;
private static final JAXBContext jaxbCtx;
static {
try {
jaxbCtx = JAXBContext.newInstance(ClusterMetadata.class);
} catch (final JAXBException je) {
throw new RuntimeException(je);
}
}
@XmlJavaTypeAdapter(NodeIdentifierAdapter.class)
public NodeIdentifier getPrimaryNodeId() {
return primaryNodeId;
}
public void setPrimaryNodeId(final NodeIdentifier primaryNodeId) {
this.primaryNodeId = primaryNodeId;
}
}
}

View File

@ -1,403 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.flow.impl;
import java.util.Collections;
import java.util.Date;
import java.util.Set;
import java.util.Timer;
import java.util.TimerTask;
import java.util.concurrent.CopyOnWriteArraySet;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.nifi.cluster.flow.ClusterDataFlow;
import org.apache.nifi.cluster.flow.DaoException;
import org.apache.nifi.cluster.flow.DataFlowDao;
import org.apache.nifi.cluster.flow.DataFlowManagementService;
import org.apache.nifi.cluster.flow.PersistedFlowState;
import org.apache.nifi.cluster.protocol.ClusterManagerProtocolSender;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.cluster.protocol.StandardDataFlow;
import org.apache.nifi.cluster.protocol.message.FlowRequestMessage;
import org.apache.nifi.cluster.protocol.message.FlowResponseMessage;
import org.apache.nifi.logging.NiFiLog;
import org.apache.nifi.util.FormatUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Implements FlowManagementService interface. The service tries to keep the cluster's flow current with regards to the available nodes.
*
* The instance may be configured with a retrieval delay, which will reduce the number of retrievals performed by the service at the expense of increasing the chances that the service will not be able
* to provide a current flow to the caller.
*
* By default, the service will try to update the flow as quickly as possible. Configuring a delay enables a less aggressive retrieval strategy. Specifically, the eligible retrieval time is reset
* every time the flow state is set to STALE. If the state is set to UNKNOWN or CURRENT, then the flow will not be retrieved.
*
*/
public class DataFlowManagementServiceImpl implements DataFlowManagementService {
/*
* Developer Note:
*
* This class maintains an ExecutorService and a Runnable.
* Although the class is not externally threadsafe, its internals are protected to
* accommodate multithread access between the ExecutorServer and the Runnable.
*/
private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(DataFlowManagementServiceImpl.class));
private final DataFlowDao flowDao;
private final ClusterManagerProtocolSender sender;
private final Set<NodeIdentifier> nodeIds = new CopyOnWriteArraySet<>();
private final AtomicBoolean stopRequested = new AtomicBoolean(false);
private final AtomicLong lastRetrievalTime = new AtomicLong(-1);
private Timer flowRetriever;
private long retrievableAfterTime = 0L;
private AtomicInteger retrievalDelaySeconds = new AtomicInteger(0);
private final TimingReentrantLock resourceLock = new TimingReentrantLock(new ReentrantLock());
public DataFlowManagementServiceImpl(final DataFlowDao flowDao, final ClusterManagerProtocolSender sender) {
if (flowDao == null) {
throw new IllegalArgumentException("Flow DAO may not be null.");
} else if (sender == null) {
throw new IllegalArgumentException("Cluster Manager Protocol Sender may not be null.");
}
this.flowDao = flowDao;
this.sender = sender;
}
@Override
public void start() {
if (isRunning()) {
throw new IllegalArgumentException("Instance is already running.");
}
// reset stop requested
stopRequested.set(false);
// setup flow retreiver timer
flowRetriever = new Timer("Flow Management Service", /* is daemon */ true);
flowRetriever.schedule(new FlowRetrieverTimerTask(), 0, 500);
}
@Override
public boolean isRunning() {
return (flowRetriever != null);
}
@Override
public void stop() {
if (isRunning() == false) {
throw new IllegalArgumentException("Instance is already stopped.");
}
// record stop request
stopRequested.set(true);
flowRetriever.cancel();
flowRetriever = null;
}
@Override
public ClusterDataFlow loadDataFlow() throws DaoException {
resourceLock.lock();
try {
return flowDao.loadDataFlow();
} finally {
resourceLock.unlock("loadDataFlow");
}
}
@Override
public void updatePrimaryNode(final NodeIdentifier nodeId) {
resourceLock.lock();
try {
final ClusterDataFlow existingClusterDataFlow = flowDao.loadDataFlow();
final StandardDataFlow dataFlow;
final byte[] controllerServiceBytes;
final byte[] reportingTaskBytes;
if (existingClusterDataFlow == null) {
dataFlow = null;
controllerServiceBytes = new byte[0];
reportingTaskBytes = new byte[0];
} else {
dataFlow = existingClusterDataFlow.getDataFlow();
controllerServiceBytes = existingClusterDataFlow.getControllerServices();
reportingTaskBytes = existingClusterDataFlow.getReportingTasks();
}
flowDao.saveDataFlow(new ClusterDataFlow(dataFlow, nodeId, controllerServiceBytes, reportingTaskBytes));
} finally {
resourceLock.unlock("updatePrimaryNode");
}
}
@Override
public void updateControllerServices(final byte[] controllerServiceBytes) throws DaoException {
resourceLock.lock();
try {
final ClusterDataFlow existingClusterDataFlow = flowDao.loadDataFlow();
final StandardDataFlow dataFlow;
final byte[] reportingTaskBytes;
final NodeIdentifier nodeId;
if (existingClusterDataFlow == null) {
dataFlow = null;
nodeId = null;
reportingTaskBytes = new byte[0];
} else {
dataFlow = existingClusterDataFlow.getDataFlow();
nodeId = existingClusterDataFlow.getPrimaryNodeId();
reportingTaskBytes = existingClusterDataFlow.getReportingTasks();
}
flowDao.saveDataFlow(new ClusterDataFlow(dataFlow, nodeId, controllerServiceBytes, reportingTaskBytes));
} finally {
resourceLock.unlock("updateControllerServices");
}
}
@Override
public void updateReportingTasks(final byte[] reportingTaskBytes) throws DaoException {
resourceLock.lock();
try {
final ClusterDataFlow existingClusterDataFlow = flowDao.loadDataFlow();
final StandardDataFlow dataFlow;
final byte[] controllerServiceBytes;
final NodeIdentifier nodeId;
if (existingClusterDataFlow == null) {
dataFlow = null;
nodeId = null;
controllerServiceBytes = null;
} else {
dataFlow = existingClusterDataFlow.getDataFlow();
nodeId = existingClusterDataFlow.getPrimaryNodeId();
controllerServiceBytes = existingClusterDataFlow.getControllerServices();
}
flowDao.saveDataFlow(new ClusterDataFlow(dataFlow, nodeId, controllerServiceBytes, reportingTaskBytes));
} finally {
resourceLock.unlock("updateControllerServices");
}
}
@Override
public PersistedFlowState getPersistedFlowState() {
resourceLock.lock();
try {
return flowDao.getPersistedFlowState();
} finally {
resourceLock.unlock("getPersistedFlowState");
}
}
@Override
public boolean isFlowCurrent() {
return PersistedFlowState.CURRENT == getPersistedFlowState();
}
@Override
public void setPersistedFlowState(final PersistedFlowState flowState) {
// lock to ensure state change and retrievable time update are atomic
resourceLock.lock();
try {
flowDao.setPersistedFlowState(flowState);
if (PersistedFlowState.STALE == flowState) {
retrievableAfterTime = new Date().getTime() + (getRetrievalDelaySeconds() * 1000);
} else if (PersistedFlowState.UNKNOWN == flowState || PersistedFlowState.CURRENT == flowState) {
retrievableAfterTime = Long.MAX_VALUE;
}
} finally {
resourceLock.unlock("setPersistedFlowState");
}
}
@Override
public Set<NodeIdentifier> getNodeIds() {
return Collections.unmodifiableSet(nodeIds);
}
@Override
public void setNodeIds(final Set<NodeIdentifier> nodeIds) {
if (nodeIds == null) {
throw new IllegalArgumentException("Node IDs may not be null.");
}
resourceLock.lock();
try {
if (this.nodeIds.equals(nodeIds)) {
return;
}
this.nodeIds.clear();
this.nodeIds.addAll(nodeIds);
} finally {
resourceLock.unlock("setNodeIds");
}
}
@Override
public int getRetrievalDelaySeconds() {
return retrievalDelaySeconds.get();
}
@Override
public void setRetrievalDelay(final String retrievalDelay) {
this.retrievalDelaySeconds.set((int) FormatUtils.getTimeDuration(retrievalDelay, TimeUnit.SECONDS));
}
public ClusterManagerProtocolSender getSender() {
return sender;
}
public long getLastRetrievalTime() {
return lastRetrievalTime.get();
}
/**
* A timer task for issuing FlowRequestMessage messages to nodes to retrieve an updated flow.
*/
private class FlowRetrieverTimerTask extends TimerTask {
@Override
public void run() {
resourceLock.lock();
try {
// if flow is current, then we're done
if (isFlowCurrent()) {
return;
}
} catch (final Exception ex) {
logger.info("Encountered exception checking if flow is current caused by " + ex, ex);
} finally {
resourceLock.unlock("FlowRetrieverTimerTask - isFlowCurrent");
}
final FlowRequestMessage request = new FlowRequestMessage();
for (final NodeIdentifier nodeId : getNodeIds()) {
try {
// setup request
request.setNodeId(nodeId);
// record request time
final long requestSentTime = new Date().getTime();
resourceLock.lock();
try {
// sanity checks before making request
if (stopRequested.get()) { // did we receive a stop request
logger.debug("Stopping runnable prematurely because a request to stop was issued.");
return;
} else if (requestSentTime < retrievableAfterTime) {
/*
* Retrievable after time was updated while obtaining
* the lock, so try again later
*/
return;
}
} finally {
resourceLock.unlock("FlowRetrieverTimerTask - check stopRequested");
}
// send request
final FlowResponseMessage response = sender.requestFlow(request);
resourceLock.lock();
try {
// check if the retrieved flow is still valid
if (requestSentTime > retrievableAfterTime) {
logger.info("Saving retrieved flow.");
final StandardDataFlow dataFlow = response.getDataFlow();
final ClusterDataFlow existingClusterDataFlow = flowDao.loadDataFlow();
final ClusterDataFlow currentClusterDataFlow;
if (existingClusterDataFlow == null) {
currentClusterDataFlow = new ClusterDataFlow(dataFlow, null, new byte[0], new byte[0]);
} else {
currentClusterDataFlow = new ClusterDataFlow(dataFlow, existingClusterDataFlow.getPrimaryNodeId(),
existingClusterDataFlow.getControllerServices(), existingClusterDataFlow.getReportingTasks());
}
flowDao.saveDataFlow(currentClusterDataFlow);
flowDao.setPersistedFlowState(PersistedFlowState.CURRENT);
lastRetrievalTime.set(new Date().getTime());
}
/*
* Retrievable after time was updated while requesting
* the flow, so try again later.
*/
} finally {
resourceLock.unlock("FlowRetrieverTimerTask - saveDataFlow");
}
} catch (final Throwable t) {
logger.info("Encountered exception retrieving flow from node " + nodeId + " caused by " + t, t);
}
}
}
}
private static class TimingReentrantLock {
private final Lock lock;
private static final Logger logger = LoggerFactory.getLogger("dataFlowManagementService.lock");
private final ThreadLocal<Long> lockTime = new ThreadLocal<>();
public TimingReentrantLock(final Lock lock) {
this.lock = lock;
}
public void lock() {
lock.lock();
lockTime.set(System.nanoTime());
}
public void unlock(final String task) {
final long nanosLocked = System.nanoTime() - lockTime.get();
lock.unlock();
final long millisLocked = TimeUnit.MILLISECONDS.convert(nanosLocked, TimeUnit.NANOSECONDS);
if (millisLocked > 100L) {
logger.debug("Lock held for {} milliseconds for task: {}", millisLocked, task);
}
}
}
}

View File

@ -1,142 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.manager;
import java.util.List;
import java.util.Set;
import org.apache.nifi.cluster.event.Event;
import org.apache.nifi.cluster.manager.exception.IllegalNodeDeletionException;
import org.apache.nifi.cluster.manager.exception.IllegalNodeDisconnectionException;
import org.apache.nifi.cluster.manager.exception.IllegalNodeReconnectionException;
import org.apache.nifi.cluster.manager.exception.NodeDisconnectionException;
import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
import org.apache.nifi.cluster.node.Node;
import org.apache.nifi.cluster.node.Node.Status;
import org.apache.nifi.cluster.protocol.ConnectionRequest;
import org.apache.nifi.cluster.protocol.ConnectionResponse;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.remote.cluster.NodeInformant;
import org.apache.nifi.reporting.BulletinRepository;
/**
* Defines the interface for a ClusterManager. The cluster manager is a threadsafe centralized manager for a cluster. Members of a cluster are nodes. A member becomes a node by issuing a connection
* request to the manager. The manager maintains the set of nodes. Nodes may be disconnected, reconnected, and deleted.
*
* Nodes are responsible for sending heartbeats to the manager to indicate their liveliness. A manager may disconnect a node if it does not receive a heartbeat within a configurable time period. A
* cluster manager instance may be configured with how often to monitor received heartbeats (getHeartbeatMonitoringIntervalSeconds()) and the maximum time that may elapse between node heartbeats
* before disconnecting the node (getMaxHeartbeatGapSeconds()).
*
* Since only a single node may execute isolated processors, the cluster manager maintains the notion of a primary node. The primary node is chosen at cluster startup and retains the role until a user
* requests a different node to be the primary node.
*
*/
public interface ClusterManager extends NodeInformant {
/**
* @param statuses the statuses of the nodes
* @return the set of nodes
*/
Set<Node> getNodes(Status... statuses);
/**
* @param nodeId node identifier
* @return returns the node with the given identifier or null if node does not exist
*/
Node getNode(String nodeId);
/**
* @param statuses statuses
* @return the set of node identifiers with the given node status
*/
Set<NodeIdentifier> getNodeIds(Status... statuses);
/**
* Deletes the node with the given node identifier. If the given node is the primary node, then a subsequent request may be made to the manager to set a new primary node.
*
* @param nodeId the node identifier
* @param userDn the Distinguished Name of the user requesting the node be deleted from the cluster
*
* @throws UnknownNodeException if the node does not exist
* @throws IllegalNodeDeletionException if the node is not in a disconnected state
*/
void deleteNode(String nodeId, String userDn) throws UnknownNodeException, IllegalNodeDeletionException;
/**
* Requests a connection to the cluster.
*
* @param request the request
*
* @return the response
*/
ConnectionResponse requestConnection(ConnectionRequest request);
/**
* Services reconnection requests for a given node. If the node indicates reconnection failure, then the node will be set to disconnected. Otherwise, a reconnection request will be sent to the
* node, initiating the connection handshake.
*
* @param nodeId a node identifier
* @param userDn the Distinguished Name of the user requesting the reconnection
*
* @throws UnknownNodeException if the node does not exist
* @throws IllegalNodeReconnectionException if the node is not disconnected
*/
void requestReconnection(String nodeId, String userDn) throws UnknownNodeException, IllegalNodeReconnectionException;
/**
* Requests the node with the given identifier be disconnected.
*
* @param nodeId the node identifier
* @param userDn the Distinguished Name of the user requesting the disconnection
*
* @throws UnknownNodeException if the node does not exist
* @throws IllegalNodeDisconnectionException if the node cannot be disconnected due to the cluster's state (e.g., node is last connected node or node is primary)
* @throws UnknownNodeException if the node does not exist
* @throws IllegalNodeDisconnectionException if the node is not disconnected
* @throws NodeDisconnectionException if the disconnection failed
*/
void requestDisconnection(String nodeId, String userDn) throws UnknownNodeException, IllegalNodeDisconnectionException, NodeDisconnectionException;
/**
* @return the time in seconds to wait between successive executions of heartbeat monitoring
*/
int getHeartbeatMonitoringIntervalSeconds();
/**
* @return the maximum time in seconds that is allowed between successive heartbeats of a node before disconnecting the node
*/
int getMaxHeartbeatGapSeconds();
/**
* Returns a list of node events for the node with the given identifier. The events will be returned in order of most recent to least recent according to the creation date of the event.
*
* @param nodeId the node identifier
*
* @return the list of events or an empty list if no node exists with the given identifier
*/
List<Event> getNodeEvents(final String nodeId);
/**
* @return the primary node of the cluster or null if no primary node exists
*/
Node getPrimaryNode();
/**
* @return the bulletin repository
*/
BulletinRepository getBulletinRepository();
}

View File

@ -1,129 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.manager;
import java.net.URI;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.nifi.cluster.manager.exception.ConnectingNodeMutableRequestException;
import org.apache.nifi.cluster.manager.exception.DisconnectedNodeMutableRequestException;
import org.apache.nifi.cluster.manager.exception.NoConnectedNodesException;
import org.apache.nifi.cluster.manager.exception.NoResponseFromNodesException;
import org.apache.nifi.cluster.manager.exception.SafeModeMutableRequestException;
import org.apache.nifi.cluster.manager.exception.UriConstructionException;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
/**
* Extends the ClusterManager interface to define how requests issued to the cluster manager are federated to the nodes. Specifically, the HTTP protocol is used for communicating requests to the
* cluster manager and to the nodes.
*
*/
public interface HttpClusterManager extends ClusterManager {
/**
* Federates the HTTP request to all connected nodes in the cluster. The given URI's host and port will not be used and instead will be adjusted for each node's host and port. The node URIs are
* guaranteed to be constructed before issuing any requests, so if a UriConstructionException is thrown, then it is guaranteed that no request was issued.
*
* @param method the HTTP method (e.g., GET, POST, PUT, DELETE, HEAD)
* @param uri the base request URI (up to, but not including, the query string)
* @param parameters the request parameters
* @param headers the request headers
*
* @return the client response
*
* @throws NoConnectedNodesException if no nodes are connected as results of the request
* @throws NoResponseFromNodesException if no response could be obtained
* @throws UriConstructionException if there was an issue constructing the URIs tailored for each individual node
* @throws ConnectingNodeMutableRequestException if the request was a PUT, POST, DELETE and a node is connecting to the cluster
* @throws DisconnectedNodeMutableRequestException if the request was a PUT, POST, DELETE and a node is disconnected from the cluster
* @throws SafeModeMutableRequestException if the request was a PUT, POST, DELETE and a the cluster is in safe mode
*/
NodeResponse applyRequest(String method, URI uri, Map<String, List<String>> parameters, Map<String, String> headers)
throws NoConnectedNodesException, NoResponseFromNodesException, UriConstructionException, ConnectingNodeMutableRequestException,
DisconnectedNodeMutableRequestException, SafeModeMutableRequestException;
/**
* Federates the HTTP request to the nodes specified. The given URI's host and port will not be used and instead will be adjusted for each node's host and port. The node URIs are guaranteed to be
* constructed before issuing any requests, so if a UriConstructionException is thrown, then it is guaranteed that no request was issued.
*
* @param method the HTTP method (e.g., GET, POST, PUT, DELETE, HEAD)
* @param uri the base request URI (up to, but not including, the query string)
* @param parameters the request parameters
* @param headers the request headers
* @param nodeIdentifiers the NodeIdentifier for each node that the request should be replaced to
*
* @return the client response
*
* @throws NoConnectedNodesException if no nodes are connected as results of the request
* @throws NoResponseFromNodesException if no response could be obtained
* @throws UriConstructionException if there was an issue constructing the URIs tailored for each individual node
* @throws ConnectingNodeMutableRequestException if the request was a PUT, POST, DELETE and a node is connecting to the cluster
* @throws DisconnectedNodeMutableRequestException if the request was a PUT, POST, DELETE and a node is disconnected from the cluster
* @throws SafeModeMutableRequestException if the request was a PUT, POST, DELETE and a the cluster is in safe mode
*/
NodeResponse applyRequest(String method, URI uri, Map<String, List<String>> parameters, Map<String, String> headers,
Set<NodeIdentifier> nodeIdentifiers)
throws NoConnectedNodesException, NoResponseFromNodesException, UriConstructionException, ConnectingNodeMutableRequestException,
DisconnectedNodeMutableRequestException, SafeModeMutableRequestException;
/**
* Federates the HTTP request to all connected nodes in the cluster. The given URI's host and port will not be used and instead will be adjusted for each node's host and port. The node URIs are
* guaranteed to be constructed before issuing any requests, so if a UriConstructionException is thrown, then it is guaranteed that no request was issued.
*
* @param method the HTTP method (e.g., GET, POST, PUT, DELETE, HEAD)
* @param uri the base request URI (up to, but not including, the query string)
* @param entity the HTTP request entity
* @param headers the request headers
*
* @return the client response
*
* @throws NoConnectedNodesException if no nodes are connected as results of the request
* @throws NoResponseFromNodesException if no response could be obtained
* @throws UriConstructionException if there was an issue constructing the URIs tailored for each individual node
* @throws ConnectingNodeMutableRequestException if the request was a PUT, POST, DELETE and a node is connecting to the cluster
* @throws DisconnectedNodeMutableRequestException if the request was a PUT, POST, DELETE and a node is disconnected from the cluster
* @throws SafeModeMutableRequestException if the request was a PUT, POST, DELETE and a the cluster is in safe mode
*/
NodeResponse applyRequest(String method, URI uri, Object entity, Map<String, String> headers)
throws NoConnectedNodesException, NoResponseFromNodesException, UriConstructionException, ConnectingNodeMutableRequestException,
DisconnectedNodeMutableRequestException, SafeModeMutableRequestException;
/**
* Federates the HTTP request to the nodes specified. The given URI's host and port will not be used and instead will be adjusted for each node's host and port. The node URIs are guaranteed to be
* constructed before issuing any requests, so if a UriConstructionException is thrown, then it is guaranteed that no request was issued.
*
* @param method the HTTP method (e.g., GET, POST, PUT, DELETE, HEAD)
* @param uri the base request URI (up to, but not including, the query string)
* @param entity the HTTP request entity
* @param headers the request headers
* @param nodeIdentifiers the NodeIdentifier for each node that the request should be replaced to
*
* @return the client response
*
* @throws NoConnectedNodesException if no nodes are connected as results of the request
* @throws NoResponseFromNodesException if no response could be obtained
* @throws UriConstructionException if there was an issue constructing the URIs tailored for each individual node
* @throws ConnectingNodeMutableRequestException if the request was a PUT, POST, DELETE and a node is connecting to the cluster
* @throws DisconnectedNodeMutableRequestException if the request was a PUT, POST, DELETE and a node is disconnected from the cluster
* @throws SafeModeMutableRequestException if the request was a PUT, POST, DELETE and a the cluster is in safe mode
*/
NodeResponse applyRequest(String method, URI uri, Object entity, Map<String, String> headers, Set<NodeIdentifier> nodeIdentifiers)
throws NoConnectedNodesException, NoResponseFromNodesException, UriConstructionException, ConnectingNodeMutableRequestException,
DisconnectedNodeMutableRequestException, SafeModeMutableRequestException;
}

View File

@ -1,84 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.manager;
import java.net.URI;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.nifi.cluster.manager.exception.UriConstructionException;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
/**
* A service for managing the replication of requests to nodes. It is up to the implementing class to decide if requests are sent concurrently or serially.
*
* Clients must call start() and stop() to initialize and shutdown the instance. The instance must be started before issuing any replication requests.
*
*/
public interface HttpRequestReplicator {
/**
* Starts the instance for replicating requests. Start may only be called if the instance is not running.
*/
void start();
/**
* Stops the instance from replicating requests. Stop may only be called if the instance is running.
*/
void stop();
/**
* @return true if the instance is started; false otherwise.
*/
boolean isRunning();
/**
* Requests are sent to each node in the cluster. If the request results in an exception, then the NodeResourceResponse will contain the exception.
*
* HTTP DELETE and OPTIONS methods must supply an empty parameters map or else and IllegalArgumentException is thrown.
*
* @param nodeIds the node identifiers
* @param method the HTTP method (e.g., GET, POST, PUT, DELETE, HEAD, OPTIONS)
* @param uri the base request URI (up to, but not including, the query string)
* @param parameters any request parameters
* @param headers any HTTP headers
*
* @return the set of node responses
*
* @throws UriConstructionException if a request for a node failed to be constructed from the given prototype URI. If thrown, it is guaranteed that no request was sent.
*/
Set<NodeResponse> replicate(Set<NodeIdentifier> nodeIds, String method, URI uri, Map<String, List<String>> parameters, Map<String, String> headers) throws UriConstructionException;
/**
* Requests are sent to each node in the cluster. If the request results in an exception, then the NodeResourceResponse will contain the exception.
*
* HTTP DELETE, GET, HEAD, and OPTIONS methods will throw an IllegalArgumentException if used.
*
* @param nodeIds the node identifiers
* @param method the HTTP method (e.g., POST, PUT)
* @param uri the base request URI (up to, but not including, the query string)
* @param entity an entity
* @param headers any HTTP headers
*
* @return the set of node responses
*
* @throws UriConstructionException if a request for a node failed to be constructed from the given prototype URI. If thrown, it is guaranteed that no request was sent.
*/
Set<NodeResponse> replicate(Set<NodeIdentifier> nodeIds, String method, URI uri, Object entity, Map<String, String> headers) throws UriConstructionException;
}

View File

@ -1,41 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.manager;
import java.net.URI;
import java.util.Map;
import java.util.Set;
import org.apache.nifi.cluster.node.Node.Status;
/**
* Maps a HTTP response to a node status.
*
*/
public interface HttpResponseMapper {
/**
* Maps a HTTP response to a node response and the corresponding node status.
*
* @param requestURI the original request URI
* @param nodeResponses a set of node resource responses
*
* @return a map associating the node response to the node status
*/
Map<NodeResponse, Status> map(URI requestURI, Set<NodeResponse> nodeResponses);
}

View File

@ -131,9 +131,6 @@ public class NodeResponse {
return requestUri;
}
/**
* @return the HTTP response status code
*/
public int getStatus() {
if (hasThrowable()) {
/*
@ -151,49 +148,25 @@ public class NodeResponse {
}
}
/**
* @return true if the response status is 2xx, false otherwise.
*/
public boolean is2xx() {
final int statusCode = getStatus();
return (200 <= statusCode && statusCode <= 299);
}
/**
* @return true if the response status is 5xx, false otherwise.
*/
public boolean is5xx() {
final int statusCode = getStatus();
return (500 <= statusCode && statusCode <= 599);
}
/**
* Returns null if hasThrowable() is true; otherwise the client's response is returned.
*
* The ClientResponse's input stream can only be read once.
*
* @return the client's response
*/
public ClientResponse getClientResponse() {
return clientResponse;
}
/**
* If this node response has been merged returns the updated entity, otherwise null. Also returns null if hasThrowable() is true. The intent of this method is to support getting the response
* entity when it was already consumed during the merge operation. In this case the client response rom getClientResponse() will not support a getEntity(...) or getEntityInputStream() call.
*
* @return If this node response has been merged returns the updated entity, otherwise null. Also returns null if hasThrowable() is true
*/
public Entity getUpdatedEntity() {
return updatedEntity;
}
/**
* Creates a Response by mapping the ClientResponse values to it. Since the ClientResponse's input stream can only be read once, this method should only be called once. Furthermore, the caller
* should not have already read the ClientResponse's input stream.
*
* @return the response
*/
public Response getResponse() {
// if the response encapsulates a throwable, then the input stream is never read and the below warning is irrelevant
if (hasCreatedResponse && !hasThrowable()) {
@ -203,20 +176,11 @@ public class NodeResponse {
return createResponse();
}
/**
* Returns the throwable or null if no throwable exists.
*
* @return the throwable or null if no throwable exists
*/
public Throwable getThrowable() {
return throwable;
}
/**
* Returns true if a throwable was thrown and a response was not able to be created; false otherwise.
*
* @return true if a throwable was thrown and a response was not able to be created; false otherwise
*/
public boolean hasThrowable() {
return getThrowable() != null;
}
@ -313,10 +277,10 @@ public class NodeResponse {
public String toString() {
final StringBuilder sb = new StringBuilder();
sb.append("NodeResponse[nodeUri=").append(nodeId.getApiAddress()).append(":").append(nodeId.getApiPort()).append(",")
.append("method=").append(httpMethod)
.append(",URI=").append(requestUri)
.append(",ResponseCode=").append(getStatus())
.append(",Duration=").append(TimeUnit.MILLISECONDS.convert(requestDurationNanos, TimeUnit.NANOSECONDS)).append(" ms]");
.append("method=").append(httpMethod)
.append(",URI=").append(requestUri)
.append(",ResponseCode=").append(getStatus())
.append(",Duration=").append(TimeUnit.MILLISECONDS.convert(requestDurationNanos, TimeUnit.NANOSECONDS)).append(" ms]");
return sb.toString();
}
}

View File

@ -22,7 +22,7 @@ import org.apache.nifi.cluster.protocol.NodeIdentifier;
*
*/
public class BlockedByFirewallException extends ClusterException {
private static final long serialVersionUID = 1L;
private final NodeIdentifier nodeId;
private final boolean isExistingNode;

View File

@ -1,39 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.manager.exception;
/**
* Represents the exceptional case when the primary role cannot be assigned to a node because the node is ineligible for the role.
*
*/
public class IneligiblePrimaryNodeException extends IllegalClusterStateException {
public IneligiblePrimaryNodeException() {
}
public IneligiblePrimaryNodeException(String msg) {
super(msg);
}
public IneligiblePrimaryNodeException(Throwable cause) {
super(cause);
}
public IneligiblePrimaryNodeException(String msg, Throwable cause) {
super(msg, cause);
}
}

View File

@ -1,39 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.manager.exception;
/**
* Represents the exceptional case when the cluster is unable to update the primary role of a node.
*
*/
public class PrimaryRoleAssignmentException extends IllegalClusterStateException {
public PrimaryRoleAssignmentException() {
}
public PrimaryRoleAssignmentException(String msg) {
super(msg);
}
public PrimaryRoleAssignmentException(Throwable cause) {
super(cause);
}
public PrimaryRoleAssignmentException(String msg, Throwable cause) {
super(msg, cause);
}
}

View File

@ -1,39 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.manager.exception;
/**
* Represents the exceptional case when a HTTP request that may change a node's dataflow is to be replicated while the cluster is in safe mode.
*
*/
public class SafeModeMutableRequestException extends MutableRequestException {
public SafeModeMutableRequestException() {
}
public SafeModeMutableRequestException(String msg) {
super(msg);
}
public SafeModeMutableRequestException(Throwable cause) {
super(cause);
}
public SafeModeMutableRequestException(String msg, Throwable cause) {
super(msg, cause);
}
}

View File

@ -1,146 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.manager.impl;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.nifi.action.Action;
import org.apache.nifi.admin.service.AuditService;
import org.apache.nifi.controller.status.ProcessGroupStatus;
import org.apache.nifi.events.EventReporter;
import org.apache.nifi.history.History;
import org.apache.nifi.provenance.ProvenanceEventBuilder;
import org.apache.nifi.provenance.ProvenanceEventRecord;
import org.apache.nifi.provenance.ProvenanceEventRepository;
import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
import org.apache.nifi.provenance.search.Query;
import org.apache.nifi.provenance.search.QuerySubmission;
import org.apache.nifi.provenance.search.SearchableField;
import org.apache.nifi.reporting.EventAccess;
public class ClusteredEventAccess implements EventAccess {
private final WebClusterManager clusterManager;
private final AuditService auditService;
public ClusteredEventAccess(final WebClusterManager clusterManager, final AuditService auditService) {
this.clusterManager = clusterManager;
this.auditService = auditService;
}
@Override
public ProcessGroupStatus getControllerStatus() {
return new ProcessGroupStatus();
}
@Override
public List<ProvenanceEventRecord> getProvenanceEvents(long arg0, int arg1) throws IOException {
return new ArrayList<>();
}
@Override
public ProvenanceEventRepository getProvenanceRepository() {
// NCM doesn't have provenance events, because it doesn't process FlowFiles.
// So we just use a Provenance Event Repository that does nothing.
return new ProvenanceEventRepository() {
@Override
public void close() throws IOException {
}
@Override
public ProvenanceEventRecord getEvent(long eventId) throws IOException {
return null;
}
@Override
public List<ProvenanceEventRecord> getEvents(long startEventId, int maxEvents) throws IOException {
return new ArrayList<>();
}
@Override
public Long getMaxEventId() {
return null;
}
@Override
public List<SearchableField> getSearchableAttributes() {
return new ArrayList<>();
}
@Override
public List<SearchableField> getSearchableFields() {
return new ArrayList<>();
}
@Override
public void registerEvent(final ProvenanceEventRecord event) {
}
@Override
public void registerEvents(final Iterable<ProvenanceEventRecord> events) {
}
@Override
public ComputeLineageSubmission retrieveLineageSubmission(final String submissionId) {
return null;
}
@Override
public QuerySubmission retrieveQuerySubmission(final String submissionId) {
return null;
}
@Override
public ComputeLineageSubmission submitExpandChildren(final long eventId) {
return null;
}
@Override
public ComputeLineageSubmission submitExpandParents(final long eventId) {
return null;
}
@Override
public ComputeLineageSubmission submitLineageComputation(final String flowFileUuid) {
return null;
}
@Override
public QuerySubmission submitQuery(final Query query) {
return null;
}
@Override
public ProvenanceEventBuilder eventBuilder() {
return null;
}
@Override
public void initialize(EventReporter eventReporter) throws IOException {
}
};
}
@Override
public List<Action> getFlowChanges(int firstActionId, int maxActions) {
final History history = auditService.getActions(firstActionId, maxActions);
return new ArrayList<>(history.getActions());
}
}

View File

@ -1,217 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.manager.impl;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import org.apache.nifi.attribute.expression.language.PreparedQuery;
import org.apache.nifi.attribute.expression.language.Query;
import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.controller.ControllerServiceLookup;
import org.apache.nifi.controller.service.ControllerServiceProvider;
import org.apache.nifi.controller.status.PortStatus;
import org.apache.nifi.controller.status.ProcessGroupStatus;
import org.apache.nifi.controller.status.ProcessorStatus;
import org.apache.nifi.controller.status.RemoteProcessGroupStatus;
import org.apache.nifi.events.BulletinFactory;
import org.apache.nifi.reporting.Bulletin;
import org.apache.nifi.reporting.BulletinRepository;
import org.apache.nifi.reporting.ComponentType;
import org.apache.nifi.reporting.EventAccess;
import org.apache.nifi.reporting.ReportingContext;
import org.apache.nifi.reporting.Severity;
public class ClusteredReportingContext implements ReportingContext {
private final EventAccess eventAccess;
private final BulletinRepository bulletinRepository;
private final ControllerServiceProvider serviceProvider;
private final Map<PropertyDescriptor, String> properties;
private final Map<PropertyDescriptor, PreparedQuery> preparedQueries;
private final StateManager stateManager;
public ClusteredReportingContext(final EventAccess eventAccess, final BulletinRepository bulletinRepository, final Map<PropertyDescriptor, String> properties,
final ControllerServiceProvider serviceProvider, final StateManager stateManager) {
this.eventAccess = eventAccess;
this.bulletinRepository = bulletinRepository;
this.properties = Collections.unmodifiableMap(properties);
this.serviceProvider = serviceProvider;
this.stateManager = stateManager;
preparedQueries = new HashMap<>();
for (final Map.Entry<PropertyDescriptor, String> entry : properties.entrySet()) {
final PropertyDescriptor desc = entry.getKey();
String value = entry.getValue();
if (value == null) {
value = desc.getDefaultValue();
}
final PreparedQuery pq = Query.prepare(value);
preparedQueries.put(desc, pq);
}
}
@Override
public EventAccess getEventAccess() {
return eventAccess;
}
@Override
public BulletinRepository getBulletinRepository() {
return bulletinRepository;
}
@Override
public Bulletin createBulletin(final String category, final Severity severity, final String message) {
return BulletinFactory.createBulletin(category, severity.name(), message);
}
@Override
public Bulletin createBulletin(final String componentId, final String category, final Severity severity, final String message) {
final ProcessGroupStatus rootGroupStatus = eventAccess.getControllerStatus();
final String groupId = findGroupId(rootGroupStatus, componentId);
final String componentName = findComponentName(rootGroupStatus, componentId);
final ComponentType componentType = findComponentType(rootGroupStatus, componentId);
return BulletinFactory.createBulletin(groupId, componentId, componentType, componentName, category, severity.name(), message);
}
@Override
public Map<PropertyDescriptor, String> getProperties() {
return Collections.unmodifiableMap(properties);
}
@Override
public PropertyValue getProperty(final PropertyDescriptor property) {
final String configuredValue = properties.get(property);
return new StandardPropertyValue(configuredValue == null ? property.getDefaultValue() : configuredValue, serviceProvider, preparedQueries.get(property));
}
@Override
public ControllerServiceLookup getControllerServiceLookup() {
return serviceProvider;
}
String findGroupId(final ProcessGroupStatus groupStatus, final String componentId) {
for (final ProcessorStatus procStatus : groupStatus.getProcessorStatus()) {
if (procStatus.getId().equals(componentId)) {
return groupStatus.getId();
}
}
for (final PortStatus portStatus : groupStatus.getInputPortStatus()) {
if (portStatus.getId().equals(componentId)) {
return groupStatus.getId();
}
}
for (final PortStatus portStatus : groupStatus.getOutputPortStatus()) {
if (portStatus.getId().equals(componentId)) {
return groupStatus.getId();
}
}
for (final ProcessGroupStatus childGroup : groupStatus.getProcessGroupStatus()) {
final String groupId = findGroupId(childGroup, componentId);
if (groupId != null) {
return groupId;
}
}
return null;
}
private ComponentType findComponentType(final ProcessGroupStatus groupStatus, final String componentId) {
for (final ProcessorStatus procStatus : groupStatus.getProcessorStatus()) {
if (procStatus.getId().equals(componentId)) {
return ComponentType.PROCESSOR;
}
}
for (final PortStatus portStatus : groupStatus.getInputPortStatus()) {
if (portStatus.getId().equals(componentId)) {
return ComponentType.INPUT_PORT;
}
}
for (final PortStatus portStatus : groupStatus.getOutputPortStatus()) {
if (portStatus.getId().equals(componentId)) {
return ComponentType.OUTPUT_PORT;
}
}
for (final RemoteProcessGroupStatus remoteStatus : groupStatus.getRemoteProcessGroupStatus()) {
if (remoteStatus.getId().equals(componentId)) {
return ComponentType.REMOTE_PROCESS_GROUP;
}
}
for (final ProcessGroupStatus childGroup : groupStatus.getProcessGroupStatus()) {
final ComponentType type = findComponentType(childGroup, componentId);
if (type != null) {
return type;
}
}
final ControllerService service = serviceProvider.getControllerService(componentId);
if (service != null) {
return ComponentType.CONTROLLER_SERVICE;
}
return null;
}
private String findComponentName(final ProcessGroupStatus groupStatus, final String componentId) {
for (final ProcessorStatus procStatus : groupStatus.getProcessorStatus()) {
if (procStatus.getId().equals(componentId)) {
return procStatus.getName();
}
}
for (final PortStatus portStatus : groupStatus.getInputPortStatus()) {
if (portStatus.getId().equals(componentId)) {
return groupStatus.getName();
}
}
for (final PortStatus portStatus : groupStatus.getOutputPortStatus()) {
if (portStatus.getId().equals(componentId)) {
return groupStatus.getName();
}
}
for (final ProcessGroupStatus childGroup : groupStatus.getProcessGroupStatus()) {
final String componentName = findComponentName(childGroup, componentId);
if (componentName != null) {
return componentName;
}
}
return null;
}
@Override
public StateManager getStateManager() {
return stateManager;
}
}

View File

@ -1,512 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.manager.impl;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.CompletionService;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorCompletionService;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import javax.ws.rs.HttpMethod;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.MultivaluedMap;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.cluster.coordination.http.StandardHttpResponseMerger;
import org.apache.nifi.cluster.manager.HttpRequestReplicator;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.manager.exception.UriConstructionException;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.logging.NiFiLog;
import org.apache.nifi.util.FormatUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.sun.jersey.api.client.Client;
import com.sun.jersey.api.client.ClientResponse;
import com.sun.jersey.api.client.UniformInterfaceException;
import com.sun.jersey.api.client.WebResource;
import com.sun.jersey.api.client.config.ClientConfig;
import com.sun.jersey.api.client.filter.GZIPContentEncodingFilter;
import com.sun.jersey.core.util.MultivaluedMapImpl;
/**
* An implementation of the <code>HttpRequestReplicator</code> interface. This implementation parallelizes the node HTTP requests using the given <code>ExecutorService</code> instance. Individual
* requests may have connection and read timeouts set, which may be set during instance construction. Otherwise, the default is not to timeout.
*
* If a node protocol scheme is provided during construction, then all requests will be replicated using the given scheme. If null is provided as the scheme (the default), then the requests will be
* replicated using the scheme of the original URI.
*
* Clients must call start() and stop() to initialize and shutdown the instance. The instance must be started before issuing any replication requests.
*
*/
public class HttpRequestReplicatorImpl implements HttpRequestReplicator {
// defaults
private static final int DEFAULT_SHUTDOWN_REPLICATOR_SECONDS = 30;
// logger
private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(HttpRequestReplicatorImpl.class));
// final members
private final Client client; // the client to use for issuing requests
private final int numThreads; // number of threads to use for request replication
private final int connectionTimeoutMs; // connection timeout per node request
private final int readTimeoutMs; // read timeout per node request
// members
private ExecutorService executorService;
private int shutdownReplicatorSeconds = DEFAULT_SHUTDOWN_REPLICATOR_SECONDS;
// guarded by synchronized method access in support of multithreaded replication
private String nodeProtocolScheme = null;
/**
* Creates an instance. The connection timeout and read timeout will be infinite.
*
* @param numThreads the number of threads to use when parallelizing requests
* @param client a client for making requests
*/
public HttpRequestReplicatorImpl(final int numThreads, final Client client) {
this(numThreads, client, "0 sec", "0 sec");
}
/**
* Creates an instance.
*
* @param numThreads the number of threads to use when parallelizing requests
* @param client a client for making requests
* @param connectionTimeout the connection timeout specified in milliseconds
* @param readTimeout the read timeout specified in milliseconds
*/
public HttpRequestReplicatorImpl(final int numThreads, final Client client, final String connectionTimeout, final String readTimeout) {
if (numThreads <= 0) {
throw new IllegalArgumentException("The number of threads must be greater than zero.");
} else if (client == null) {
throw new IllegalArgumentException("Client may not be null.");
}
this.numThreads = numThreads;
this.client = client;
this.connectionTimeoutMs = (int) FormatUtils.getTimeDuration(connectionTimeout, TimeUnit.MILLISECONDS);
this.readTimeoutMs = (int) FormatUtils.getTimeDuration(readTimeout, TimeUnit.MILLISECONDS);
client.getProperties().put(ClientConfig.PROPERTY_CONNECT_TIMEOUT, connectionTimeoutMs);
client.getProperties().put(ClientConfig.PROPERTY_READ_TIMEOUT, readTimeoutMs);
client.getProperties().put(ClientConfig.PROPERTY_FOLLOW_REDIRECTS, Boolean.TRUE);
}
@Override
public void start() {
if (isRunning()) {
throw new IllegalStateException("Instance is already started.");
}
executorService = Executors.newFixedThreadPool(numThreads);
}
@Override
public boolean isRunning() {
return executorService != null && !executorService.isShutdown();
}
@Override
public void stop() {
if (!isRunning()) {
throw new IllegalStateException("Instance is already stopped.");
}
// shutdown executor service
try {
if (getShutdownReplicatorSeconds() <= 0) {
executorService.shutdownNow();
} else {
executorService.shutdown();
}
executorService.awaitTermination(getShutdownReplicatorSeconds(), TimeUnit.SECONDS);
} catch (final InterruptedException ex) {
Thread.currentThread().interrupt();
} finally {
if (executorService.isTerminated()) {
logger.info("HTTP Request Replicator has been terminated successfully.");
} else {
logger.warn("HTTP Request Replicator has not terminated properly. There exists an uninterruptable thread that will take an indeterminate amount of time to stop.");
}
}
}
/**
* Sets the protocol scheme to use when issuing requests to nodes.
*
* @param nodeProtocolScheme the scheme. Valid values are "http", "https", or null. If null is specified, then the scheme of the originating request is used when replicating that request.
*/
public synchronized void setNodeProtocolScheme(final String nodeProtocolScheme) {
if (StringUtils.isNotBlank(nodeProtocolScheme)) {
if (!"http".equalsIgnoreCase(nodeProtocolScheme) && !"https".equalsIgnoreCase(nodeProtocolScheme)) {
throw new IllegalArgumentException("Node Protocol Scheme must be either HTTP or HTTPS");
}
}
this.nodeProtocolScheme = nodeProtocolScheme;
}
public synchronized String getNodeProtocolScheme() {
return nodeProtocolScheme;
}
private synchronized String getNodeProtocolScheme(final URI uri) {
// if we are not configured to use a protocol scheme, then use the uri's scheme
if (StringUtils.isBlank(nodeProtocolScheme)) {
return uri.getScheme();
}
return nodeProtocolScheme;
}
public int getConnectionTimeoutMs() {
return connectionTimeoutMs;
}
public int getReadTimeoutMs() {
return readTimeoutMs;
}
public int getShutdownReplicatorSeconds() {
return shutdownReplicatorSeconds;
}
public void setShutdownReplicatorSeconds(int shutdownReplicatorSeconds) {
this.shutdownReplicatorSeconds = shutdownReplicatorSeconds;
}
@Override
public Set<NodeResponse> replicate(final Set<NodeIdentifier> nodeIds, final String method,
final URI uri, final Map<String, List<String>> parameters, final Map<String, String> headers)
throws UriConstructionException {
if (nodeIds == null) {
throw new IllegalArgumentException("Node IDs may not be null.");
} else if (method == null) {
throw new IllegalArgumentException("HTTP method may not be null.");
} else if (uri == null) {
throw new IllegalArgumentException("URI may not be null.");
} else if (parameters == null) {
throw new IllegalArgumentException("Parameters may not be null.");
} else if (headers == null) {
throw new IllegalArgumentException("HTTP headers map may not be null.");
}
return replicateHelper(nodeIds, method, getNodeProtocolScheme(uri), uri.getPath(), parameters, /* entity */ null, headers);
}
@Override
public Set<NodeResponse> replicate(final Set<NodeIdentifier> nodeIds, final String method, final URI uri,
final Object entity, final Map<String, String> headers) throws UriConstructionException {
if (nodeIds == null) {
throw new IllegalArgumentException("Node IDs may not be null.");
} else if (method == null) {
throw new IllegalArgumentException("HTTP method may not be null.");
} else if (method.equalsIgnoreCase(HttpMethod.DELETE) || method.equalsIgnoreCase(HttpMethod.GET) || method.equalsIgnoreCase(HttpMethod.HEAD) || method.equalsIgnoreCase(HttpMethod.OPTIONS)) {
throw new IllegalArgumentException("HTTP (DELETE | GET | HEAD | OPTIONS) requests cannot have a body containing an entity.");
} else if (uri == null) {
throw new IllegalArgumentException("URI may not be null.");
} else if (entity == null) {
throw new IllegalArgumentException("Entity may not be null.");
} else if (headers == null) {
throw new IllegalArgumentException("HTTP headers map may not be null.");
}
return replicateHelper(nodeIds, method, getNodeProtocolScheme(uri), uri.getPath(), /* parameters */ null, entity, headers);
}
private Set<NodeResponse> replicateHelper(final Set<NodeIdentifier> nodeIds, final String method, final String scheme,
final String path, final Map<String, List<String>> parameters, final Object entity, final Map<String, String> headers)
throws UriConstructionException {
if (nodeIds.isEmpty()) {
return new HashSet<>(); // return quickly for trivial case
}
final CompletionService<NodeResponse> completionService = new ExecutorCompletionService<>(executorService);
// keeps track of future requests so that failed requests can be tied back to the failing node
final Collection<NodeHttpRequestFutureWrapper> futureNodeHttpRequests = new ArrayList<>();
// construct the URIs for the nodes
final Map<NodeIdentifier, URI> uriMap = new HashMap<>();
try {
for (final NodeIdentifier nodeId : nodeIds) {
final URI nodeUri = new URI(scheme, null, nodeId.getApiAddress(), nodeId.getApiPort(), path, /* query */ null, /* fragment */ null);
uriMap.put(nodeId, nodeUri);
}
} catch (final URISyntaxException use) {
throw new UriConstructionException(use);
}
// submit the requests to the nodes
for (final Map.Entry<NodeIdentifier, URI> entry : uriMap.entrySet()) {
final NodeIdentifier nodeId = entry.getKey();
final URI nodeUri = entry.getValue();
final NodeHttpRequestCallable callable = (entity == null)
? new NodeHttpRequestCallable(nodeId, method, nodeUri, parameters, headers)
: new NodeHttpRequestCallable(nodeId, method, nodeUri, entity, headers);
futureNodeHttpRequests.add(new NodeHttpRequestFutureWrapper(nodeId, method, nodeUri, completionService.submit(callable)));
}
// get the node responses
final Set<NodeResponse> result = new HashSet<>();
for (int i = 0; i < nodeIds.size(); i++) {
// keeps track of the original request information in case we receive an exception
NodeHttpRequestFutureWrapper futureNodeHttpRequest = null;
try {
// get the future resource response for the node
final Future<NodeResponse> futureNodeResourceResponse = completionService.take();
// find the original request by comparing the submitted future with the future returned by the completion service
for (final NodeHttpRequestFutureWrapper futureNodeHttpRequestElem : futureNodeHttpRequests) {
if (futureNodeHttpRequestElem.getFuture() == futureNodeResourceResponse) {
futureNodeHttpRequest = futureNodeHttpRequestElem;
}
}
// try to retrieve the node response and add to result
final NodeResponse nodeResponse = futureNodeResourceResponse.get();
result.add(nodeResponse);
} catch (final InterruptedException | ExecutionException ex) {
logger.warn("Node request for " + futureNodeHttpRequest.getNodeId() + " encountered exception: " + ex, ex);
// create node response with the thrown exception and add to result
final NodeResponse nodeResponse = new NodeResponse(
futureNodeHttpRequest.getNodeId(), futureNodeHttpRequest.getHttpMethod(), futureNodeHttpRequest.getRequestUri(), ex);
result.add(nodeResponse);
}
}
if (logger.isDebugEnabled()) {
NodeResponse min = null;
NodeResponse max = null;
long nanosSum = 0L;
int nanosAdded = 0;
for (final NodeResponse response : result) {
final long requestNanos = response.getRequestDuration(TimeUnit.NANOSECONDS);
final long minNanos = (min == null) ? -1 : min.getRequestDuration(TimeUnit.NANOSECONDS);
final long maxNanos = (max == null) ? -1 : max.getRequestDuration(TimeUnit.NANOSECONDS);
if (requestNanos < minNanos || minNanos < 0L) {
min = response;
}
if (requestNanos > maxNanos || maxNanos < 0L) {
max = response;
}
if (requestNanos >= 0L) {
nanosSum += requestNanos;
nanosAdded++;
}
}
final StringBuilder sb = new StringBuilder();
sb.append("Node Responses for ").append(method).append(" ").append(path).append(":\n");
for (final NodeResponse response : result) {
sb.append(response).append("\n");
}
final long averageNanos = (nanosAdded == 0) ? -1L : nanosSum / nanosAdded;
final long averageMillis = (averageNanos < 0) ? averageNanos : TimeUnit.MILLISECONDS.convert(averageNanos, TimeUnit.NANOSECONDS);
logger.debug("For {} {}, minimum response time = {}, max = {}, average = {} ms",
method, path, min, max, averageMillis);
logger.debug(sb.toString());
}
return result;
}
/**
* Wraps a future node response with info from originating request. This coupling allows for futures that encountered exceptions to be linked back to the failing node and better reported.
*/
private class NodeHttpRequestFutureWrapper {
private final NodeIdentifier nodeId;
private final String httpMethod;
private final URI requestUri;
private final Future<NodeResponse> future;
public NodeHttpRequestFutureWrapper(final NodeIdentifier nodeId, final String httpMethod,
final URI requestUri, final Future<NodeResponse> future) {
if (nodeId == null) {
throw new IllegalArgumentException("Node ID may not be null.");
} else if (StringUtils.isBlank(httpMethod)) {
throw new IllegalArgumentException("Http method may not be null or empty.");
} else if (requestUri == null) {
throw new IllegalArgumentException("Request URI may not be null.");
} else if (future == null) {
throw new IllegalArgumentException("Future may not be null.");
}
this.nodeId = nodeId;
this.httpMethod = httpMethod;
this.requestUri = requestUri;
this.future = future;
}
public NodeIdentifier getNodeId() {
return nodeId;
}
public String getHttpMethod() {
return httpMethod;
}
public URI getRequestUri() {
return requestUri;
}
public Future<NodeResponse> getFuture() {
return future;
}
}
/**
* A Callable for making an HTTP request to a single node and returning its response.
*/
private class NodeHttpRequestCallable implements Callable<NodeResponse> {
private final NodeIdentifier nodeId;
private final String method;
private final URI uri;
private final Object entity;
private final Map<String, List<String>> parameters = new HashMap<>();
private final Map<String, String> headers = new HashMap<>();
private NodeHttpRequestCallable(final NodeIdentifier nodeId, final String method,
final URI uri, final Object entity, final Map<String, String> headers) {
this.nodeId = nodeId;
this.method = method;
this.uri = uri;
this.entity = entity;
this.headers.putAll(headers);
}
private NodeHttpRequestCallable(final NodeIdentifier nodeId, final String method,
final URI uri, final Map<String, List<String>> parameters, final Map<String, String> headers) {
this.nodeId = nodeId;
this.method = method;
this.uri = uri;
this.entity = null;
this.parameters.putAll(parameters);
this.headers.putAll(headers);
}
@Override
public NodeResponse call() {
try {
// create and send the request
final WebResource.Builder resourceBuilder = getResourceBuilder();
final String requestId = headers.get("x-nifi-request-id");
final long startNanos = System.nanoTime();
final ClientResponse clientResponse;
if (HttpMethod.DELETE.equalsIgnoreCase(method)) {
clientResponse = resourceBuilder.delete(ClientResponse.class);
} else if (HttpMethod.GET.equalsIgnoreCase(method)) {
clientResponse = resourceBuilder.get(ClientResponse.class);
} else if (HttpMethod.HEAD.equalsIgnoreCase(method)) {
clientResponse = resourceBuilder.head();
} else if (HttpMethod.OPTIONS.equalsIgnoreCase(method)) {
clientResponse = resourceBuilder.options(ClientResponse.class);
} else if (HttpMethod.POST.equalsIgnoreCase(method)) {
clientResponse = resourceBuilder.post(ClientResponse.class);
} else if (HttpMethod.PUT.equalsIgnoreCase(method)) {
clientResponse = resourceBuilder.put(ClientResponse.class);
} else {
throw new IllegalArgumentException("HTTP Method '" + method + "' not supported for request replication.");
}
// create and return the response
return new NodeResponse(nodeId, method, uri, clientResponse, System.nanoTime() - startNanos, requestId);
} catch (final UniformInterfaceException | IllegalArgumentException t) {
return new NodeResponse(nodeId, method, uri, t);
}
}
private WebResource.Builder getResourceBuilder() {
// convert parameters to a more convenient data structure
final MultivaluedMap<String, String> map = new MultivaluedMapImpl();
map.putAll(parameters);
// create the resource
WebResource resource = client.resource(uri);
if (new StandardHttpResponseMerger().isResponseInterpreted(uri, method)) {
resource.addFilter(new GZIPContentEncodingFilter(false));
}
// set the parameters as either query parameters or as request body
final WebResource.Builder builder;
if (HttpMethod.DELETE.equalsIgnoreCase(method) || HttpMethod.HEAD.equalsIgnoreCase(method) || HttpMethod.GET.equalsIgnoreCase(method) || HttpMethod.OPTIONS.equalsIgnoreCase(method)) {
resource = resource.queryParams(map);
builder = resource.getRequestBuilder();
} else {
if (entity == null) {
builder = resource.entity(map);
} else {
builder = resource.entity(entity);
}
}
// set headers
boolean foundContentType = false;
for (final Map.Entry<String, String> entry : headers.entrySet()) {
builder.header(entry.getKey(), entry.getValue());
if (entry.getKey().equalsIgnoreCase("content-type")) {
foundContentType = true;
}
}
// set default content type
if (!foundContentType) {
// set default content type
builder.type(MediaType.APPLICATION_FORM_URLENCODED);
}
return builder;
}
}
}

View File

@ -1,82 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.manager.impl;
import java.net.URI;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import org.apache.nifi.cluster.manager.HttpResponseMapper;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.node.Node;
import org.apache.nifi.cluster.node.Node.Status;
import org.apache.nifi.logging.NiFiLog;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Determines the status of nodes based on their HTTP response codes.
*
* The algorithm is as follows.
*
* If any HTTP responses were 2XX, then disconnect non-2XX responses. This is because 2XX may have changed a node's flow.
*
* If no 2XX responses were received, then the node's flow has not changed. Instead of disconnecting everything, we only disconnect the nodes with internal errors, i.e., 5XX responses.
*
*/
public class HttpResponseMapperImpl implements HttpResponseMapper {
private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(HttpResponseMapperImpl.class));
@Override
public Map<NodeResponse, Status> map(final URI requestURI, final Set<NodeResponse> nodeResponses) {
final Map<NodeResponse, Status> result = new HashMap<>();
// check if any responses were 2XX
boolean found2xx = false;
for (final NodeResponse nodeResponse : nodeResponses) {
if (nodeResponse.is2xx()) {
found2xx = true;
break;
}
}
// determine the status of each node
for (final NodeResponse nodeResponse : nodeResponses) {
final Node.Status status;
if (found2xx) {
// disconnect nodes with non-2XX responses
status = nodeResponse.is2xx()
? Node.Status.CONNECTED
: Node.Status.DISCONNECTED;
} else {
// disconnect nodes with 5XX responses or exception
status = nodeResponse.is5xx()
? Node.Status.DISCONNECTED
: Node.Status.CONNECTED;
}
result.put(nodeResponse, status);
}
return result;
}
}

View File

@ -1,264 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.manager.impl;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;
import java.util.stream.Collectors;
import org.apache.nifi.cluster.coordination.ClusterCoordinator;
import org.apache.nifi.cluster.coordination.node.DisconnectionCode;
import org.apache.nifi.cluster.coordination.node.NodeConnectionState;
import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus;
import org.apache.nifi.cluster.flow.DataFlowManagementService;
import org.apache.nifi.cluster.node.Node;
import org.apache.nifi.cluster.node.Node.Status;
import org.apache.nifi.cluster.protocol.ClusterManagerProtocolSender;
import org.apache.nifi.cluster.protocol.ConnectionRequest;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.cluster.protocol.message.NodeStatusChangeMessage;
import org.apache.nifi.reporting.Severity;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class WebClusterManagerCoordinator implements ClusterCoordinator {
private static final Logger logger = LoggerFactory.getLogger(WebClusterManagerCoordinator.class);
private static final AtomicLong nodeStatusIdGenerator = new AtomicLong(0L);
private final WebClusterManager clusterManager;
private final ClusterManagerProtocolSender protocolSender;
private final DataFlowManagementService dfmService;
public WebClusterManagerCoordinator(final WebClusterManager clusterManager, final ClusterManagerProtocolSender protocolSender, final DataFlowManagementService dfmService) {
this.clusterManager = clusterManager;
this.protocolSender = protocolSender;
this.dfmService = dfmService;
}
@Override
public void requestNodeConnect(final NodeIdentifier nodeId) {
final Node node = clusterManager.getRawNode(nodeId.getId());
if (node == null) {
final ConnectionRequest connectionRequest = new ConnectionRequest(nodeId);
clusterManager.requestConnection(connectionRequest);
} else {
updateNodeStatus(nodeId, new NodeConnectionStatus(DisconnectionCode.NOT_YET_CONNECTED, "Requesting that Node Connect to the Cluster"));
clusterManager.requestReconnection(nodeId.getId(), "Anonymous");
}
}
@Override
public void finishNodeConnection(final NodeIdentifier nodeId) {
final boolean updated = updateNodeStatus(nodeId, new NodeConnectionStatus(NodeConnectionState.CONNECTED));
if (!updated) {
logger.error("Attempting to Finish Node Connection but could not find Node with Identifier {}", nodeId);
}
}
@Override
public void requestNodeDisconnect(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode, final String explanation) {
try {
clusterManager.requestDisconnection(nodeId, false, explanation);
if (disconnectionCode == DisconnectionCode.LACK_OF_HEARTBEAT) {
final Node node = clusterManager.getRawNode(nodeId.getId());
if (node != null) {
updateNodeStatus(node, Status.DISCONNECTED, true);
}
}
} catch (final Exception e) {
logger.error("Failed to request node {} disconnect from cluster due to {}", nodeId, e);
logger.error("", e);
}
}
@Override
public void disconnectionRequestedByNode(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode, final String explanation) {
updateNodeStatus(nodeId, new NodeConnectionStatus(disconnectionCode, explanation));
final Severity severity;
switch (disconnectionCode) {
case STARTUP_FAILURE:
case MISMATCHED_FLOWS:
case UNKNOWN:
severity = Severity.ERROR;
break;
default:
severity = Severity.INFO;
break;
}
reportEvent(nodeId, severity, "Node disconnected from cluster due to " + explanation);
}
@Override
public NodeConnectionStatus getConnectionStatus(final NodeIdentifier nodeId) {
final Node node = clusterManager.getNode(nodeId.getId());
if (node == null) {
return null;
}
final Status status = node.getStatus();
final NodeConnectionState connectionState = NodeConnectionState.valueOf(status.name());
return new NodeConnectionStatus(connectionState, node.getConnectionRequestedTimestamp());
}
@Override
public Set<NodeIdentifier> getNodeIdentifiers(final NodeConnectionState state) {
final Status status = Status.valueOf(state.name());
final Set<Node> nodes = clusterManager.getNodes(status);
return nodes.stream()
.map(node -> node.getNodeId())
.collect(Collectors.toSet());
}
@Override
public boolean isBlockedByFirewall(final String hostname) {
return clusterManager.isBlockedByFirewall(hostname);
}
@Override
public void reportEvent(final NodeIdentifier nodeId, final Severity severity, final String event) {
final String messagePrefix = nodeId == null ? "" : nodeId.getApiAddress() + ":" + nodeId.getApiPort() + " -- ";
switch (severity) {
case INFO:
logger.info(messagePrefix + event);
break;
case WARNING:
logger.warn(messagePrefix + event);
break;
case ERROR:
logger.error(messagePrefix + event);
break;
}
clusterManager.reportEvent(nodeId, severity, messagePrefix + event);
}
@Override
public void setPrimaryNode(final NodeIdentifier nodeId) {
clusterManager.setPrimaryNodeId(nodeId);
}
@Override
public NodeIdentifier getNodeIdentifier(final String uuid) {
final Node node = clusterManager.getNode(uuid);
return node == null ? null : node.getNodeId();
}
/**
* Updates the status of the node with the given ID to the given status and returns <code>true</code>
* if successful, <code>false</code> if no node exists with the given ID
*
* @param nodeId the ID of the node whose status is changed
* @param status the new status of the node
* @return <code>true</code> if the node exists and is updated, <code>false</code> if the node does not exist
*/
private boolean updateNodeStatus(final NodeIdentifier nodeId, final NodeConnectionStatus status) {
final long statusUpdateId = nodeStatusIdGenerator.incrementAndGet();
final Node node = clusterManager.getRawNode(nodeId.getId());
if (node == null) {
return false;
}
final Status nodeStatus = Status.valueOf(status.getState().name());
final Status oldStatus = node.setStatus(nodeStatus);
if (nodeStatus != oldStatus) {
final Set<NodeIdentifier> nodesToNotify = clusterManager.getNodes(Status.CONNECTED, Status.CONNECTING).stream()
.map(toNotify -> toNotify.getNodeId())
.collect(Collectors.toSet());
final NodeStatusChangeMessage message = new NodeStatusChangeMessage();
message.setNodeId(nodeId);
message.setNodeConnectionStatus(status);
// TODO: When this is sent from one node to another, we need to ensure that we check the current
// 'revision number' on the node and include that as the Update ID because we need a way to indicate
// which status change event occurred first. I.e., when the status of a node is updated on any node
// that is not the elected leader, we need to ensure that our nodeStatusIdGenerator also is updated.
message.setStatusUpdateIdentifier(statusUpdateId);
protocolSender.notifyNodeStatusChange(nodesToNotify, message);
}
dfmService.setNodeIds(getNodeIdentifiers(NodeConnectionState.CONNECTED));
return true;
}
/**
* Updates the status of the given node to the given new status. This method exists only because the NCM currently handles
* some of the status changing and we want it to call into this coordinator instead to change the status.
*
* @param rawNode the node whose status should be updated
* @param nodeStatus the new status of the node
*/
void updateNodeStatus(final Node rawNode, final Status nodeStatus) {
// TODO: Remove this method when NCM is removed
updateNodeStatus(rawNode, nodeStatus, false);
}
/**
* Updates the status of the given node to the given new status. This method exists only because the NCM currently handles
* some of the status changing and we want it to call into this coordinator instead to change the status.
*
* @param rawNode the node whose status should be updated
* @param nodeStatus the new status of the node
* @param heartbeatDisconnect indicates whether or not the node is being disconnected due to lack of heartbeat
*/
void updateNodeStatus(final Node rawNode, final Status nodeStatus, final boolean heartbeatDisconnect) {
// TODO: Remove this method when NCM is removed.
final long statusUpdateId = nodeStatusIdGenerator.incrementAndGet();
final Status oldStatus;
if (heartbeatDisconnect) {
oldStatus = rawNode.setHeartbeatDisconnection();
} else {
oldStatus = rawNode.setStatus(nodeStatus);
}
if (nodeStatus != oldStatus) {
final Set<NodeIdentifier> nodesToNotify = clusterManager.getNodes(Status.CONNECTED, Status.CONNECTING).stream()
.map(toNotify -> toNotify.getNodeId())
.collect(Collectors.toSet());
final NodeStatusChangeMessage message = new NodeStatusChangeMessage();
message.setNodeId(rawNode.getNodeId());
message.setNodeConnectionStatus(new NodeConnectionStatus(NodeConnectionState.valueOf(nodeStatus.name())));
message.setStatusUpdateIdentifier(statusUpdateId);
protocolSender.notifyNodeStatusChange(nodesToNotify, message);
dfmService.setNodeIds(getNodeIdentifiers(NodeConnectionState.CONNECTED));
}
}
@Override
public Map<NodeConnectionState, List<NodeIdentifier>> getConnectionStates() {
final Set<Node> nodes = clusterManager.getNodes();
final Map<NodeConnectionState, List<NodeIdentifier>> connectionStatusMap = nodes.stream()
.map(node -> node.getNodeId())
.collect(Collectors.groupingBy(nodeId -> getConnectionStatus(nodeId).getState()));
return connectionStatusMap;
}
}

Some files were not shown because too many files have changed in this diff Show More