NIFI-1678:

- Started refactoring heartbeating mechanism, using a new package: org.apache.nifi.cluster.coordination
- Added configuration options for ZooKeeper username & password for heartbeat management. Also addressed issue where nodes that were previously disconnected were asked to disconnect upon restart
- Ensure that ACL is set properly when creating heartbeat node. Removed unused ControllerStartupFailureMessage.java
- Changed ZooKeeper ACL's so that container nodes that would not be sensitive are wide open and removed the usage of username & password when communicating with ZooKeeper. This was done specifically because username/password combination is considered a 'testing' feature that should not be used in production and is not supported by Apache Curator
- Refactored CuratorHeartbeatMonitor into an abstract heartbeat monitor that is responsible for processing heartbeats and CuratorHeartbeatMonitor that is responsible for retrieving heartbeat information
- Refactored so that heartbeats are sent to Cluster Coordinator directly instead of to ZooKeeper. ZooKeeper is used to know which node is the cluster coordinator but heartbeats to the Cluster Coordinator provide additional information about the nodes.
- Started refactoring heartbeating mechanism, using a new package: org.apache.nifi.cluster.coordination
- Added configuration options for ZooKeeper username & password for heartbeat management. Also addressed issue where nodes that were previously disconnected were asked to disconnect upon restart
- Changed ZooKeeper ACL's so that container nodes that would not be sensitive are wide open and removed the usage of username & password when communicating with ZooKeeper. This was done specifically because username/password combination is considered a 'testing' feature that should not be used in production and is not supported by Apache Curator

NIFI-1727:
- Refactored logic for merging HTTP Requests that are federated across cluster

NIFI-1745:
- Refactoring how HTTP Requests are replicated to nodes
- Bug fixes and continuing to work on replication refactoring. Still need to handle cluster locking and revisions
- Begin work on RevisionManager
- Resolved some issues that resulted from rebase
- Fixed URIs to align with new URI's that will be used in 1.0.0
- This closes #413
This commit is contained in:
Mark Payne 2016-03-24 11:49:08 -04:00 committed by Matt Gilman
parent 3db14f58fe
commit 04c41c0654
135 changed files with 9707 additions and 3764 deletions

View File

@ -16,7 +16,9 @@
*/
package org.apache.nifi.controller;
import java.util.Set;
import java.util.Map;
import org.apache.nifi.web.Revision;
/**
* A Snippet represents a segment of the flow
@ -44,42 +46,42 @@ public interface Snippet {
/**
* @return connections in this snippet
*/
public Set<String> getConnections();
public Map<String, Revision> getConnections();
/**
* @return funnels in this snippet
*/
public Set<String> getFunnels();
public Map<String, Revision> getFunnels();
/**
* @return input ports in this snippet
*/
public Set<String> getInputPorts();
public Map<String, Revision> getInputPorts();
/**
* @return output ports in this snippet
*/
public Set<String> getOutputPorts();
public Map<String, Revision> getOutputPorts();
/**
* @return labels in this snippet
*/
public Set<String> getLabels();
public Map<String, Revision> getLabels();
/**
* @return the identifiers of all ProcessGroups in this Snippet
*/
public Set<String> getProcessGroups();
public Map<String, Revision> getProcessGroups();
/**
* @return the identifiers of all Processors in this Snippet
*/
public Set<String> getProcessors();
public Map<String, Revision> getProcessors();
/**
* @return the identifiers of all RemoteProcessGroups in this Snippet
*/
public Set<String> getRemoteProcessGroups();
public Map<String, Revision> getRemoteProcessGroups();
/**
* @return Determines if this snippet is empty

View File

@ -445,6 +445,7 @@ public class ProcessGroupStatus implements Cloneable {
merged.setOutputBytes(merged.getOutputBytes() + statusToMerge.getOutputBytes());
merged.setOutputCount(merged.getOutputCount() + statusToMerge.getOutputCount());
merged.setProcessingNanos(merged.getProcessingNanos() + statusToMerge.getProcessingNanos());
merged.setFlowFilesRemoved(merged.getFlowFilesRemoved() + statusToMerge.getFlowFilesRemoved());
// if the status to merge is invalid allow it to take precedence. whether the
// processor run status is disabled/stopped/running is part of the flow configuration

View File

@ -17,12 +17,8 @@
package org.apache.nifi.controller.status.history;
import java.util.Date;
import java.util.List;
import org.apache.nifi.controller.status.ConnectionStatus;
import org.apache.nifi.controller.status.ProcessGroupStatus;
import org.apache.nifi.controller.status.ProcessorStatus;
import org.apache.nifi.controller.status.RemoteProcessGroupStatus;
/**
* A repository for storing and retrieving components' historical status
@ -121,29 +117,4 @@ public interface ComponentStatusRepository {
* period
*/
StatusHistory getRemoteProcessGroupStatusHistory(String remoteGroupId, Date start, Date end, int preferredDataPoints);
/**
* @return a List of all {@link MetricDescriptor}s that are applicable to
* Process Groups
*/
List<MetricDescriptor<ProcessGroupStatus>> getProcessGroupMetricDescriptors();
/**
* @return a List of all {@link MetricDescriptor}s that are applicable to
* Processors
*/
List<MetricDescriptor<ProcessorStatus>> getProcessorMetricDescriptors();
/**
* @return a List of all {@link MetricDescriptor}s that are applicable to
* Remote Process Groups
*/
List<MetricDescriptor<RemoteProcessGroupStatus>> getRemoteProcessGroupMetricDescriptors();
/**
* @return a List of all {@link MetricDescriptor}s that are applicable to
* Connections
*/
List<MetricDescriptor<ConnectionStatus>> getConnectionMetricDescriptors();
}

View File

@ -17,10 +17,11 @@
package org.apache.nifi.web;
import java.io.Serializable;
import java.util.Objects;
/**
* A model object representing a revision. Equality is defined as either a
* matching version number or matching non-empty client IDs.
* A model object representing a revision. Equality is defined as matching
* component ID and either a matching version number or matching non-empty client IDs.
*
* @Immutable
* @Threadsafe
@ -37,9 +38,22 @@ public class Revision implements Serializable {
*/
private final String clientId;
/**
* the ID of the component that this revision belongs to, or <code>null</code> if
* the revision is not attached to any component but rather is attached to the entire
* data flow.
*/
private final String componentId;
@Deprecated
public Revision(Long revision, String clientId) {
this(revision, clientId, "root"); // TODO: remove this constructor. This is to bridge the gap right now
}
public Revision(Long revision, String clientId, String componentId) {
this.version = revision;
this.clientId = clientId;
this.componentId = Objects.requireNonNull(componentId);
}
public String getClientId() {
@ -50,14 +64,40 @@ public class Revision implements Serializable {
return version;
}
public String getComponentId() {
return componentId;
}
@Override
public boolean equals(final Object obj) {
if (obj == null) {
return false;
}
if (obj == this) {
return true;
}
if ((obj instanceof Revision) == false) {
return false;
}
// TODO: THIS IS FOR TESTING PURPOSES! DO NOT LET THIS GET CHECKED IN THIS WAY!!!!!!!!!!!!
if (true) {
return true;
}
Revision thatRevision = (Revision) 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 {
@ -69,6 +109,7 @@ public class Revision implements Serializable {
@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;
@ -76,6 +117,6 @@ public class Revision implements Serializable {
@Override
public String toString() {
return "[" + version + ", " + clientId + ']';
return "[" + version + ", " + clientId + ", " + componentId + ']';
}
}

View File

@ -462,7 +462,6 @@ language governing permissions and limitations under the License. -->
<nifi.zookeeper.connect.timeout>3 secs</nifi.zookeeper.connect.timeout>
<nifi.zookeeper.session.timeout>3 secs</nifi.zookeeper.session.timeout>
<nifi.zookeeper.root.node>/nifi</nifi.zookeeper.root.node>
<nifi.zookeeper.access.control>Open</nifi.zookeeper.access.control>
<!-- nifi.properties: kerberos properties -->
<nifi.kerberos.krb5.file> </nifi.kerberos.krb5.file>

View File

@ -172,7 +172,6 @@ public class NiFiProperties extends Properties {
public static final String ZOOKEEPER_CONNECT_TIMEOUT = "nifi.zookeeper.connect.timeout";
public static final String ZOOKEEPER_SESSION_TIMEOUT = "nifi.zookeeper.session.timeout";
public static final String ZOOKEEPER_ROOT_NODE = "nifi.zookeeper.root.node";
public static final String ZOOKEEPER_ACCESS_CONTROL = "nifi.zookeeper.access.control";
// cluster manager properties
public static final String CLUSTER_IS_MANAGER = "nifi.cluster.is.manager";

View File

@ -16,12 +16,16 @@
*/
package org.apache.nifi.web.api.dto;
import com.wordnik.swagger.annotations.ApiModelProperty;
import java.util.Collection;
import java.util.Map;
import java.util.Set;
import javax.xml.bind.annotation.XmlType;
import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentEntity;
import com.wordnik.swagger.annotations.ApiModelProperty;
/**
* A Controller Service that can be shared by other components
*/
@ -41,7 +45,7 @@ public class ControllerServiceDTO extends ComponentDTO {
private String customUiUrl;
private String annotationData;
private Set<ControllerServiceReferencingComponentDTO> referencingComponents;
private Set<ControllerServiceReferencingComponentEntity> referencingComponents;
private Collection<String> validationErrors;
@ -193,11 +197,11 @@ public class ControllerServiceDTO extends ComponentDTO {
@ApiModelProperty(
value = "All components referencing this controller service."
)
public Set<ControllerServiceReferencingComponentDTO> getReferencingComponents() {
public Set<ControllerServiceReferencingComponentEntity> getReferencingComponents() {
return referencingComponents;
}
public void setReferencingComponents(Set<ControllerServiceReferencingComponentDTO> referencingComponents) {
public void setReferencingComponents(Set<ControllerServiceReferencingComponentEntity> referencingComponents) {
this.referencingComponents = referencingComponents;
}

View File

@ -16,11 +16,12 @@
*/
package org.apache.nifi.web.api.dto;
import com.wordnik.swagger.annotations.ApiModelProperty;
import java.util.HashMap;
import java.util.Map;
import javax.xml.bind.annotation.XmlType;
import java.util.HashSet;
import java.util.Set;
import com.wordnik.swagger.annotations.ApiModelProperty;
/**
* The contents of a snippet of a flow.
@ -34,14 +35,14 @@ public class SnippetDTO {
private Boolean linked;
// when specified these are only considered during creation
private Set<String> processGroups = new HashSet<>();
private Set<String> remoteProcessGroups = new HashSet<>();
private Set<String> processors = new HashSet<>();
private Set<String> inputPorts = new HashSet<>();
private Set<String> outputPorts = new HashSet<>();
private Set<String> connections = new HashSet<>();
private Set<String> labels = new HashSet<>();
private Set<String> funnels = new HashSet<>();
private Map<String, RevisionDTO> processGroups = new HashMap<>();
private Map<String, RevisionDTO> remoteProcessGroups = new HashMap<>();
private Map<String, RevisionDTO> processors = new HashMap<>();
private Map<String, RevisionDTO> inputPorts = new HashMap<>();
private Map<String, RevisionDTO> outputPorts = new HashMap<>();
private Map<String, RevisionDTO> connections = new HashMap<>();
private Map<String, RevisionDTO> labels = new HashMap<>();
private Map<String, RevisionDTO> funnels = new HashMap<>();
/**
* @return id of this snippet
@ -109,11 +110,11 @@ public class SnippetDTO {
value = "The ids of the connections in this snippet. These ids will be populated within each response. They can be specified when creating a snippet. However, once a snippet "
+ "has been created its contents cannot be modified (these ids are ignored during update requests)."
)
public Set<String> getConnections() {
public Map<String, RevisionDTO> getConnections() {
return connections;
}
public void setConnections(Set<String> connections) {
public void setConnections(Map<String, RevisionDTO> connections) {
this.connections = connections;
}
@ -125,11 +126,11 @@ public class SnippetDTO {
value = "The ids of the funnels in this snippet. These ids will be populated within each response. They can be specified when creating a snippet. However, once a snippet "
+ "has been created its contents cannot be modified (these ids are ignored during update requests)."
)
public Set<String> getFunnels() {
public Map<String, RevisionDTO> getFunnels() {
return funnels;
}
public void setFunnels(Set<String> funnels) {
public void setFunnels(Map<String, RevisionDTO> funnels) {
this.funnels = funnels;
}
@ -141,11 +142,11 @@ public class SnippetDTO {
value = "The ids of the input ports in this snippet. These ids will be populated within each response. They can be specified when creating a snippet. However, once a snippet "
+ "has been created its contents cannot be modified (these ids are ignored during update requests)."
)
public Set<String> getInputPorts() {
public Map<String, RevisionDTO> getInputPorts() {
return inputPorts;
}
public void setInputPorts(Set<String> inputPorts) {
public void setInputPorts(Map<String, RevisionDTO> inputPorts) {
this.inputPorts = inputPorts;
}
@ -157,11 +158,11 @@ public class SnippetDTO {
value = "The ids of the labels in this snippet. These ids will be populated within each response. They can be specified when creating a snippet. However, once a snippet "
+ "has been created its contents cannot be modified (these ids are ignored during update requests)."
)
public Set<String> getLabels() {
public Map<String, RevisionDTO> getLabels() {
return labels;
}
public void setLabels(Set<String> labels) {
public void setLabels(Map<String, RevisionDTO> labels) {
this.labels = labels;
}
@ -173,11 +174,11 @@ public class SnippetDTO {
value = "The ids of the output ports in this snippet. These ids will be populated within each response. They can be specified when creating a snippet. However, once a snippet "
+ "has been created its contents cannot be modified (these ids are ignored during update requests)."
)
public Set<String> getOutputPorts() {
public Map<String, RevisionDTO> getOutputPorts() {
return outputPorts;
}
public void setOutputPorts(Set<String> outputPorts) {
public void setOutputPorts(Map<String, RevisionDTO> outputPorts) {
this.outputPorts = outputPorts;
}
@ -189,11 +190,11 @@ public class SnippetDTO {
value = "The ids of the process groups in this snippet. These ids will be populated within each response. They can be specified when creating a snippet. However, once a snippet "
+ "has been created its contents cannot be modified (these ids are ignored during update requests)."
)
public Set<String> getProcessGroups() {
public Map<String, RevisionDTO> getProcessGroups() {
return processGroups;
}
public void setProcessGroups(Set<String> processGroups) {
public void setProcessGroups(Map<String, RevisionDTO> processGroups) {
this.processGroups = processGroups;
}
@ -205,11 +206,11 @@ public class SnippetDTO {
value = "The ids of the processors in this snippet. These ids will be populated within each response. They can be specified when creating a snippet. However, once a snippet "
+ "has been created its contents cannot be modified (these ids are ignored during update requests)."
)
public Set<String> getProcessors() {
public Map<String, RevisionDTO> getProcessors() {
return processors;
}
public void setProcessors(Set<String> processors) {
public void setProcessors(Map<String, RevisionDTO> processors) {
this.processors = processors;
}
@ -221,11 +222,11 @@ public class SnippetDTO {
value = "The ids of the remote process groups in this snippet. These ids will be populated within each response. They can be specified when creating a snippet. However, once a snippet "
+ "has been created its contents cannot be modified (these ids are ignored during update requests)."
)
public Set<String> getRemoteProcessGroups() {
public Map<String, RevisionDTO> getRemoteProcessGroups() {
return remoteProcessGroups;
}
public void setRemoteProcessGroups(Set<String> remoteProcessGroups) {
public void setRemoteProcessGroups(Map<String, RevisionDTO> remoteProcessGroups) {
this.remoteProcessGroups = remoteProcessGroups;
}

View File

@ -23,7 +23,7 @@ import javax.xml.bind.annotation.XmlType;
/**
* The NiFi flow starting at a given Process Group.
*/
@XmlType(name = "processGroupflow")
@XmlType(name = "processGroupFlow")
public class ProcessGroupFlowDTO {
private String id;

View File

@ -0,0 +1,43 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.web.api.entity;
import javax.xml.bind.annotation.XmlRootElement;
import org.apache.nifi.web.api.dto.ControllerServiceReferencingComponentDTO;
/**
* A serialized representation of this class can be placed in the entity body of a response to the API.
* This particular entity holds a reference to component that references a controller services.
*/
@XmlRootElement(name = "controllerServiceReferencingComponentEntity")
public class ControllerServiceReferencingComponentEntity extends ComponentEntity {
private ControllerServiceReferencingComponentDTO controllerServiceReferencingComponent;
/**
* @return controller service referencing components that is being serialized
*/
public ControllerServiceReferencingComponentDTO getControllerServiceReferencingComponent() {
return controllerServiceReferencingComponent;
}
public void setControllerServiceReferencingComponent(ControllerServiceReferencingComponentDTO controllerServiceReferencingComponent) {
this.controllerServiceReferencingComponent = controllerServiceReferencingComponent;
}
}

View File

@ -17,8 +17,8 @@
package org.apache.nifi.web.api.entity;
import java.util.Set;
import javax.xml.bind.annotation.XmlRootElement;
import org.apache.nifi.web.api.dto.ControllerServiceReferencingComponentDTO;
/**
* A serialized representation of this class can be placed in the entity body of a response to the API. This particular entity holds a reference to a list of controller services referencing
@ -27,16 +27,16 @@ import org.apache.nifi.web.api.dto.ControllerServiceReferencingComponentDTO;
@XmlRootElement(name = "controllerServiceReferencingComponentsEntity")
public class ControllerServiceReferencingComponentsEntity extends Entity {
private Set<ControllerServiceReferencingComponentDTO> controllerServiceReferencingComponents;
private Set<ControllerServiceReferencingComponentEntity> controllerServiceReferencingComponents;
/**
* @return list of controller service referencing components that are being serialized
* @return set of controller service referencing components that are being serialized
*/
public Set<ControllerServiceReferencingComponentDTO> getControllerServiceReferencingComponents() {
public Set<ControllerServiceReferencingComponentEntity> getControllerServiceReferencingComponents() {
return controllerServiceReferencingComponents;
}
public void setControllerServiceReferencingComponents(Set<ControllerServiceReferencingComponentDTO> controllerServiceReferencingComponents) {
public void setControllerServiceReferencingComponents(Set<ControllerServiceReferencingComponentEntity> controllerServiceReferencingComponents) {
this.controllerServiceReferencingComponents = controllerServiceReferencingComponents;
}

View File

@ -17,13 +17,14 @@
package org.apache.nifi.web.api.entity;
import javax.xml.bind.annotation.XmlRootElement;
import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO;
/**
* A serialized representation of this class can be placed in the entity body of a request or response to or from the API. This particular entity holds a reference to a RemoteProcessGroupPortDTO.
*/
@XmlRootElement(name = "remoteProcessGroupPortEntity")
public class RemoteProcessGroupPortEntity extends Entity {
public class RemoteProcessGroupPortEntity extends ComponentEntity {
private RemoteProcessGroupPortDTO remoteProcessGroupPort;

View File

@ -17,13 +17,14 @@
package org.apache.nifi.web.api.entity;
import javax.xml.bind.annotation.XmlRootElement;
import org.apache.nifi.web.api.dto.ReportingTaskDTO;
/**
* A serialized representation of this class can be placed in the entity body of a response to the API. This particular entity holds a reference to a reporting task.
*/
@XmlRootElement(name = "reportingTaskEntity")
public class ReportingTaskEntity extends Entity {
public class ReportingTaskEntity extends ComponentEntity {
private ReportingTaskDTO reportingTask;

View File

@ -23,7 +23,7 @@ import org.apache.nifi.web.api.dto.SnippetDTO;
* A serialized representation of this class can be placed in the entity body of a request or response to or from the API. This particular entity holds a reference to a SnippetDTO.
*/
@XmlRootElement(name = "snippetEntity")
public class SnippetEntity extends Entity {
public class SnippetEntity extends ComponentEntity {
private SnippetDTO snippet;

View File

@ -94,6 +94,7 @@ public class NodeProtocolSenderImpl implements NodeProtocolSender {
}
}
@Override
public void heartbeat(final HeartbeatMessage msg, final String address) throws ProtocolException {
final String hostname;
final int port;

View File

@ -0,0 +1,62 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.protocol.message;
import javax.xml.bind.annotation.XmlRootElement;
import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
/**
* Message to indicate that the status of a node in the cluster has changed
*/
@XmlRootElement(name = "nodeStatusChange")
public class NodeStatusChangeMessage extends ProtocolMessage {
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;
}
public NodeIdentifier getNodeId() {
return nodeId;
}
public void setNodeId(NodeIdentifier nodeId) {
this.nodeId = nodeId;
}
public Long getStatusUpdateIdentifier() {
return statusUpdateId;
}
public void setStatusUpdateIdentifier(Long statusUpdateId) {
this.statusUpdateId = statusUpdateId;
}
}

View File

@ -0,0 +1,62 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.protocol.message;
import javax.xml.bind.annotation.XmlRootElement;
import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
/**
* Message to indicate that the status of a node in the cluster has changed
*/
@XmlRootElement(name = "nodeStatusChange")
public class NodeStatusChangeMessage extends ProtocolMessage {
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;
}
public NodeIdentifier getNodeId() {
return nodeId;
}
public void setNodeId(NodeIdentifier nodeId) {
this.nodeId = nodeId;
}
public Long getStatusUpdateIdentifier() {
return statusUpdateId;
}
public void setStatusUpdateIdentifier(Long statusUpdateId) {
this.statusUpdateId = statusUpdateId;
}
}

View File

@ -27,11 +27,11 @@ public abstract class ProtocolMessage {
EXCEPTION,
FLOW_REQUEST,
FLOW_RESPONSE,
HEARTBEAT,
PING,
RECONNECTION_REQUEST,
RECONNECTION_RESPONSE,
SERVICE_BROADCAST,
HEARTBEAT,
NODE_STATUS_CHANGE;
}

View File

@ -17,6 +17,8 @@
package org.apache.nifi.cluster.coordination;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.nifi.cluster.coordination.node.DisconnectionCode;
@ -87,6 +89,13 @@ public interface ClusterCoordinator {
*/
Set<NodeIdentifier> getNodeIdentifiers(NodeConnectionState state);
/**
* Returns a Map of NodeConnectionStatus to all Node Identifiers that have that status.
*
* @return the NodeConnectionStatus for each Node in the cluster, grouped by the Connection Status
*/
Map<NodeConnectionState, List<NodeIdentifier>> getConnectionStates();
/**
* Checks if the given hostname is blocked by the configured firewall, returning
* <code>true</code> if the node is blocked, <code>false</code> if the node is

View File

@ -0,0 +1,376 @@
/*
* 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

@ -0,0 +1,62 @@
/*
* 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;
import java.net.URI;
import java.util.Set;
import org.apache.nifi.cluster.manager.NodeResponse;
/**
* <p>
* Maps a set of NodeResponses to a single NodeResponse for a specific REST Endpoint.
* </p>
*
* <p>
* Implementations of this interface MUST be Thread-Safe.
* </p>
*/
public interface EndpointResponseMerger {
/**
* Indicates whether or not this EndpointResponseMapper can handle mapping responses
* for the given URI and HTTP Method
*
* @param uri the URI of the endpoint
* @param method the HTTP Method used to interact with the endpoint
*
* @return <code>true</code> if the EndpointResponseMapper can handle mapping responses
* for the endpoint described by the given URI and HTTP Method
*/
boolean canHandle(URI uri, String method);
/**
* Maps the given Node Responses to a single NodeResponse that is appropriate to return
* to the client/user
*
* @param uri the URI of the REST Endpoint
* @param method the HTTP Method used to interact with the REST Endpoint
* @param successfulResponses the responses from nodes that were successful in handling the request
* @param problematicResponses the responses from nodes that were not successful in handling the request
* @param clientResponse the response that was chosen to be returned to the client
*
* @return a NodeResponse that is appropriate to return to the client/user
*/
NodeResponse merge(URI uri, String method, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses, NodeResponse clientResponse);
}

View File

@ -0,0 +1,65 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.coordination.http;
import java.net.URI;
import java.util.Set;
import org.apache.nifi.cluster.manager.NodeResponse;
/**
* <p>
* An HttpResponseMapper is responsible for taking the responses from all nodes in a cluster
* and distilling them down to a single response that would be appropriate to respond with, to the
* user/client who made the original web requests.
* </p>
*/
public interface HttpResponseMerger {
/**
* Maps the responses from all nodes in the cluster to a single NodeResponse object that
* is appropriate to respond with
*
* @param uri the URI of the web request that was made
* @param httpMethod the HTTP Method that was used when making the request
* @param nodeResponses the responses received from the individual nodes
*
* @return a single NodeResponse that represents the response that should be returned to the user/client
*/
NodeResponse mergeResponses(URI uri, String httpMethod, Set<NodeResponse> nodeResponses);
/**
* Returns a subset (or equal set) of the given Node Responses, such that all of those returned are the responses
* that indicate that the node was unable to fulfill the request
*
* @param allResponses the responses to filter
*
* @return a subset (or equal set) of the given Node Responses, such that all of those returned are the responses
* that indicate that the node was unable to fulfill the request
*/
Set<NodeResponse> getProblematicNodeResponses(Set<NodeResponse> allResponses);
/**
* Indicates whether or not the responses from nodes for the given URI & HTTP method must be interpreted in order to merge them
*
* @param uri the URI of the request
* @param httpMethod the HTTP Method of the request
* @return <code>true</code> if the response must be interpreted, <code>false</code> otherwise
*/
boolean isResponseInterpreted(URI uri, String httpMethod);
}

View File

@ -0,0 +1,243 @@
/*
* 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;
import java.io.IOException;
import java.net.URI;
import java.util.ArrayList;
import java.util.List;
import java.util.Set;
import java.util.stream.Collectors;
import javax.ws.rs.core.StreamingOutput;
import org.apache.nifi.cluster.coordination.http.endpoints.BulletinBoardEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.ComponentStateEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.ConnectionStatusEndpiontMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.ControllerServiceEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.ControllerServiceReferenceEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.ControllerServicesEndpointMerger;
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.FlowSnippetEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.GroupStatusEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.ListFlowFilesEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.PortStatusEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.ProcessGroupEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.ProcessorEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.ProcessorStatusEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.ProcessorsEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.ProvenanceEventEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.ProvenanceQueryEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.RemoteProcessGroupEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.RemoteProcessGroupStatusEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.RemoteProcessGroupsEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.ReportingTaskEndpointMerger;
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.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());
endpointMergers.add(new GroupStatusEndpointMerger());
endpointMergers.add(new ProcessorStatusEndpointMerger());
endpointMergers.add(new ConnectionStatusEndpiontMerger());
endpointMergers.add(new PortStatusEndpointMerger());
endpointMergers.add(new RemoteProcessGroupStatusEndpointMerger());
endpointMergers.add(new ProcessorEndpointMerger());
endpointMergers.add(new ProcessorsEndpointMerger());
endpointMergers.add(new RemoteProcessGroupEndpointMerger());
endpointMergers.add(new RemoteProcessGroupsEndpointMerger());
endpointMergers.add(new ProcessGroupEndpointMerger());
endpointMergers.add(new FlowSnippetEndpointMerger());
endpointMergers.add(new ProvenanceQueryEndpointMerger());
endpointMergers.add(new ProvenanceEventEndpointMerger());
endpointMergers.add(new ControllerServiceEndpointMerger());
endpointMergers.add(new ControllerServicesEndpointMerger());
endpointMergers.add(new ControllerServiceReferenceEndpointMerger());
endpointMergers.add(new ReportingTaskEndpointMerger());
endpointMergers.add(new ReportingTasksEndpointMerger());
endpointMergers.add(new DropRequestEndpiontMerger());
endpointMergers.add(new ListFlowFilesEndpointMerger());
endpointMergers.add(new ComponentStateEndpointMerger());
endpointMergers.add(new BulletinBoardEndpointMerger());
endpointMergers.add(new StatusHistoryEndpointMerger());
endpointMergers.add(new SystemDiagnosticsEndpointMerger());
endpointMergers.add(new CountersEndpointMerger());
}
public StandardHttpResponseMerger() {
this(null);
}
public StandardHttpResponseMerger(final WebClusterManager clusterManager) {
this.clusterManager = clusterManager;
}
@Override
public NodeResponse mergeResponses(final URI uri, final String httpMethod, final Set<NodeResponse> nodeResponses) {
final boolean hasSuccess = hasSuccessfulResponse(nodeResponses);
if (!hasSuccess) {
// If we have a response that is a 3xx, 4xx, or 5xx, then we want to choose that.
// Otherwise, it doesn't matter which one we choose. We do this because if we replicate
// a mutable request, it's possible that one node will respond with a 409, for instance, while
// others respond with a 150-Continue. We do not want to pick the 150-Continue; instead, we want
// the failed response.
final NodeResponse clientResponse = nodeResponses.stream().filter(p -> p.getStatus() > 299).findAny().orElse(nodeResponses.iterator().next());
// Drain the response from all nodes except for the 'chosen one'. This ensures that we don't
// leave data lingering on the socket and ensures that we don't consume the content of the response
// that we intend to respond with
drainResponses(nodeResponses, clientResponse);
return clientResponse;
}
// Determine which responses are successful
final Set<NodeResponse> successResponses = nodeResponses.stream().filter(p -> p.is2xx()).collect(Collectors.toSet());
final Set<NodeResponse> problematicResponses = nodeResponses.stream().filter(p -> !p.is2xx()).collect(Collectors.toSet());
// Choose any of the successful responses to be the 'chosen one'.
final NodeResponse clientResponse = successResponses.iterator().next();
final EndpointResponseMerger merger = getEndpointResponseMerger(uri, httpMethod);
if (merger == null) {
return clientResponse;
}
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) {
// Check if there are any 2xx responses
final boolean containsSuccessfulResponse = hasSuccessfulResponse(allResponses);
if (containsSuccessfulResponse) {
// If there is a 2xx response, we consider a response to be problematic if it is not 2xx
return allResponses.stream().filter(p -> !p.is2xx()).collect(Collectors.toSet());
} else {
// If no node is successful, we consider a problematic response to be only those that are 5xx
return allResponses.stream().filter(p -> p.is5xx()).collect(Collectors.toSet());
}
}
@Override
public boolean isResponseInterpreted(final URI uri, final String httpMethod) {
return getEndpointResponseMerger(uri, httpMethod) != null;
}
private static EndpointResponseMerger getEndpointResponseMerger(final URI uri, final String httpMethod) {
return endpointMergers.stream().filter(p -> p.canHandle(uri, httpMethod)).findFirst().orElse(null);
}
private boolean hasSuccessfulResponse(final Set<NodeResponse> allResponses) {
return allResponses.stream().anyMatch(p -> p.is2xx());
}
private void drainResponses(final Set<NodeResponse> responses, final NodeResponse exclude) {
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
.forEach(response -> drainResponse(response)); // drain all node responses that didn't get filtered out
}
private void drainResponse(final NodeResponse response) {
if (response.hasThrowable()) {
return;
}
try {
((StreamingOutput) response.getResponse().getEntity()).write(new NullOutputStream());
} catch (final IOException ioe) {
logger.info("Failed clearing out non-client response buffer from " + response.getNodeId() + " due to: " + ioe, ioe);
}
}
}

View File

@ -0,0 +1,99 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.coordination.http.endpoints;
import java.net.URI;
import java.util.HashMap;
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 AbstractMultiEntityEndpoint<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 Set<DtoType> dtos = getDtos(responseEntity);
final Map<String, Map<NodeIdentifier, DtoType>> dtoMap = new HashMap<>();
for (final NodeResponse nodeResponse : successfulResponses) {
final EntityType nodeResponseEntity = nodeResponse == clientResponse ? responseEntity : nodeResponse.getClientResponse().getEntity(getEntityClass());
final Set<DtoType> nodeDtos = getDtos(nodeResponseEntity);
for (final DtoType nodeDto : nodeDtos) {
final NodeIdentifier nodeId = nodeResponse.getNodeId();
Map<NodeIdentifier, DtoType> innerMap = dtoMap.get(nodeId);
if (innerMap == null) {
innerMap = new HashMap<>();
dtoMap.put(getComponentId(nodeDto), innerMap);
}
innerMap.put(nodeResponse.getNodeId(), nodeDto);
}
}
for (final DtoType dto : dtos) {
final String componentId = getComponentId(dto);
final Map<NodeIdentifier, DtoType> mergeMap = dtoMap.get(componentId);
mergeResponses(dto, mergeMap, successfulResponses, problematicResponses);
}
// create a new client response
return new NodeResponse(clientResponse, responseEntity);
}
/**
* @return the class that represents the type of Entity that is expected by this response mapper
*/
protected abstract Class<EntityType> getEntityClass();
/**
* Extracts the DTOs from the given entity
*
* @param entity the entity to extract the DTOs from
* @return the DTOs from the given entity
*/
protected abstract Set<DtoType> getDtos(EntityType entity);
/**
* Extracts the ID of the component that the DTO refers to
* @param dto the DTO to extract the ID from
* @return the ID of the component that the DTO refers to
*/
protected abstract String getComponentId(DtoType dto);
/**
* 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(final DtoType clientDto, Map<NodeIdentifier, DtoType> dtoMap, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses);
}

View File

@ -0,0 +1,40 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.coordination.http.endpoints;
import java.util.Map;
import java.util.Set;
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> {
@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);
if (selectedNodeId == null) {
throw new IllegalArgumentException("Attempted to merge Status request but could not find the appropriate Node Identifier");
}
mergeResponses(clientDto, dtoMap, selectedNodeId);
}
protected abstract void mergeResponses(DtoType clientDto, Map<NodeIdentifier, DtoType> dtoMap, NodeIdentifier selectedNodeId);
}

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 AbstractSingleEntityEndpoint<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

@ -0,0 +1,82 @@
/*
* 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.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.regex.Pattern;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
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 static final Pattern BULLETIN_BOARD_URI_PATTERN = Pattern.compile("/nifi-api/flow/bulletin-board");
@Override
public boolean canHandle(URI uri, String method) {
return "GET".equalsIgnoreCase(method) && BULLETIN_BOARD_URI_PATTERN.matcher(uri.getPath()).matches();
}
@Override
protected Class<BulletinBoardEntity> getEntityClass() {
return BulletinBoardEntity.class;
}
@Override
protected BulletinBoardDTO getDto(BulletinBoardEntity entity) {
return entity.getBulletinBoard();
}
@Override
protected void mergeResponses(BulletinBoardDTO clientDto, Map<NodeIdentifier, BulletinBoardDTO> dtoMap, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses) {
final List<BulletinDTO> bulletinDtos = new ArrayList<>();
for (final Map.Entry<NodeIdentifier, BulletinBoardDTO> entry : dtoMap.entrySet()) {
final NodeIdentifier nodeId = entry.getKey();
final BulletinBoardDTO boardDto = entry.getValue();
final String nodeAddress = nodeId.getApiAddress() + ":" + nodeId.getApiPort();
for (final BulletinDTO bulletin : boardDto.getBulletins()) {
bulletin.setNodeAddress(nodeAddress);
bulletinDtos.add(bulletin);
}
}
Collections.sort(bulletinDtos, new Comparator<BulletinDTO>() {
@Override
public int compare(final BulletinDTO o1, final BulletinDTO o2) {
final int timeComparison = o1.getTimestamp().compareTo(o2.getTimestamp());
if (timeComparison != 0) {
return timeComparison;
}
return o1.getNodeAddress().compareTo(o2.getNodeAddress());
}
});
clientDto.setBulletins(bulletinDtos);
}
}

View File

@ -0,0 +1,97 @@
/*
* 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.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.regex.Pattern;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.controller.state.SortedStateUtils;
import org.apache.nifi.web.api.dto.ComponentStateDTO;
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 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");
@Override
public boolean canHandle(URI uri, String method) {
if (!"GET".equalsIgnoreCase(method)) {
return false;
}
return PROCESSOR_STATE_URI_PATTERN.matcher(uri.getPath()).matches()
|| CONTROLLER_SERVICE_STATE_URI_PATTERN.matcher(uri.getPath()).matches()
|| REPORTING_TASK_STATE_URI_PATTERN.matcher(uri.getPath()).matches();
}
@Override
protected Class<ComponentStateEntity> getEntityClass() {
return ComponentStateEntity.class;
}
@Override
protected ComponentStateDTO getDto(ComponentStateEntity entity) {
return entity.getComponentState();
}
@Override
protected void mergeResponses(ComponentStateDTO clientDto, Map<NodeIdentifier, ComponentStateDTO> dtoMap, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses) {
List<StateEntryDTO> localStateEntries = new ArrayList<>();
int totalStateEntries = 0;
for (final Map.Entry<NodeIdentifier, ComponentStateDTO> nodeEntry : dtoMap.entrySet()) {
final ComponentStateDTO nodeComponentState = nodeEntry.getValue();
final NodeIdentifier nodeId = nodeEntry.getKey();
final String nodeAddress = nodeId.getApiAddress() + ":" + nodeId.getApiPort();
final StateMapDTO nodeLocalStateMap = nodeComponentState.getLocalState();
if (nodeLocalStateMap.getState() != null) {
totalStateEntries += nodeLocalStateMap.getTotalEntryCount();
for (final StateEntryDTO nodeStateEntry : nodeLocalStateMap.getState()) {
nodeStateEntry.setClusterNodeId(nodeId.getId());
nodeStateEntry.setClusterNodeAddress(nodeAddress);
localStateEntries.add(nodeStateEntry);
}
}
}
// ensure appropriate sort
Collections.sort(localStateEntries, SortedStateUtils.getEntryDtoComparator());
// sublist if necessary
if (localStateEntries.size() > SortedStateUtils.MAX_COMPONENT_STATE_ENTRIES) {
localStateEntries = localStateEntries.subList(0, SortedStateUtils.MAX_COMPONENT_STATE_ENTRIES);
}
// add all the local state entries
clientDto.getLocalState().setTotalEntryCount(totalStateEntries);
clientDto.getLocalState().setState(localStateEntries);
}
}

View File

@ -0,0 +1,74 @@
/*
* 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.ArrayList;
import java.util.Map;
import java.util.regex.Pattern;
import org.apache.nifi.cluster.manager.StatusMerger;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.web.api.dto.status.ConnectionStatusDTO;
import org.apache.nifi.web.api.dto.status.NodeConnectionStatusSnapshotDTO;
import org.apache.nifi.web.api.entity.ConnectionStatusEntity;
public class ConnectionStatusEndpiontMerger extends AbstractNodeStatusEndpoint<ConnectionStatusEntity, ConnectionStatusDTO> {
public static final Pattern CONNECTION_STATUS_URI_PATTERN = Pattern.compile("/nifi-api/flow/connections/[a-f0-9\\-]{36}/status");
@Override
public boolean canHandle(URI uri, String method) {
return "GET".equalsIgnoreCase(method) && CONNECTION_STATUS_URI_PATTERN.matcher(uri.getPath()).matches();
}
@Override
protected Class<ConnectionStatusEntity> getEntityClass() {
return ConnectionStatusEntity.class;
}
@Override
protected ConnectionStatusDTO getDto(ConnectionStatusEntity entity) {
return entity.getConnectionStatus();
}
@Override
protected void mergeResponses(ConnectionStatusDTO clientDto, Map<NodeIdentifier, ConnectionStatusDTO> dtoMap, NodeIdentifier selectedNodeId) {
final ConnectionStatusDTO mergedConnectionStatus = clientDto;
mergedConnectionStatus.setNodeSnapshots(new ArrayList<NodeConnectionStatusSnapshotDTO>());
final NodeConnectionStatusSnapshotDTO selectedNodeSnapshot = new NodeConnectionStatusSnapshotDTO();
selectedNodeSnapshot.setStatusSnapshot(clientDto.getAggregateSnapshot().clone());
selectedNodeSnapshot.setAddress(selectedNodeId.getApiAddress());
selectedNodeSnapshot.setApiPort(selectedNodeId.getApiPort());
selectedNodeSnapshot.setNodeId(selectedNodeId.getId());
mergedConnectionStatus.getNodeSnapshots().add(selectedNodeSnapshot);
// merge the other nodes
for (final Map.Entry<NodeIdentifier, ConnectionStatusDTO> entry : dtoMap.entrySet()) {
final NodeIdentifier nodeId = entry.getKey();
final ConnectionStatusDTO nodeConnectionStatus = entry.getValue();
if (nodeConnectionStatus == clientDto) {
continue;
}
StatusMerger.merge(mergedConnectionStatus, nodeConnectionStatus, nodeId.getId(), nodeId.getApiAddress(), nodeId.getApiPort());
}
}
}

View File

@ -0,0 +1,146 @@
/*
* 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.HashMap;
import java.util.Map;
import java.util.Set;
import java.util.regex.Pattern;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.controller.service.ControllerServiceState;
import org.apache.nifi.web.api.dto.ControllerServiceDTO;
import org.apache.nifi.web.api.dto.ControllerServiceReferencingComponentDTO;
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentEntity;
public class ControllerServiceEndpointMerger extends AbstractSingleEntityEndpoint<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}");
@Override
public boolean canHandle(URI uri, String method) {
if (("GET".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method)) && CONTROLLER_SERVICE_URI_PATTERN.matcher(uri.getPath()).matches()) {
return true;
} else if ("POST".equalsIgnoreCase(method) && CONTROLLER_SERVICES_URI.equals(uri.getPath())) {
return true;
}
return false;
}
@Override
protected Class<ControllerServiceEntity> getEntityClass() {
return ControllerServiceEntity.class;
}
@Override
protected ControllerServiceDTO getDto(ControllerServiceEntity entity) {
return entity.getControllerService();
}
@Override
protected void mergeResponses(ControllerServiceDTO clientDto, Map<NodeIdentifier, ControllerServiceDTO> dtoMap, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses) {
final Map<String, Set<NodeIdentifier>> validationErrorMap = new HashMap<>();
final Set<ControllerServiceReferencingComponentEntity> referencingComponents = clientDto.getReferencingComponents();
final Map<NodeIdentifier, Set<ControllerServiceReferencingComponentEntity>> nodeReferencingComponentsMap = new HashMap<>();
String state = null;
for (final Map.Entry<NodeIdentifier, ControllerServiceDTO> nodeEntry : dtoMap.entrySet()) {
final NodeIdentifier nodeId = nodeEntry.getKey();
final ControllerServiceDTO nodeControllerService = nodeEntry.getValue();
if (state == null) {
if (ControllerServiceState.DISABLING.name().equals(nodeControllerService.getState())) {
state = ControllerServiceState.DISABLING.name();
} else if (ControllerServiceState.ENABLING.name().equals(nodeControllerService.getState())) {
state = ControllerServiceState.ENABLING.name();
}
}
nodeReferencingComponentsMap.put(nodeId, nodeControllerService.getReferencingComponents());
// merge the validation errors
mergeValidationErrors(validationErrorMap, nodeId, nodeControllerService.getValidationErrors());
}
// merge the referencing components
mergeControllerServiceReferences(referencingComponents, nodeReferencingComponentsMap);
// store the 'transition' state is applicable
if (state != null) {
clientDto.setState(state);
}
// set the merged the validation errors
clientDto.setValidationErrors(normalizedMergedValidationErrors(validationErrorMap, dtoMap.size()));
}
public static void mergeControllerServiceReferences(Set<ControllerServiceReferencingComponentEntity> referencingComponents,
Map<NodeIdentifier, Set<ControllerServiceReferencingComponentEntity>> referencingComponentMap) {
final Map<String, Integer> activeThreadCounts = new HashMap<>();
final Map<String, String> states = new HashMap<>();
for (final Map.Entry<NodeIdentifier, Set<ControllerServiceReferencingComponentEntity>> nodeEntry : referencingComponentMap.entrySet()) {
final Set<ControllerServiceReferencingComponentEntity> nodeReferencingComponents = nodeEntry.getValue();
// go through all the nodes referencing components
if (nodeReferencingComponents != null) {
for (final ControllerServiceReferencingComponentEntity nodeReferencingComponentEntity : nodeReferencingComponents) {
final ControllerServiceReferencingComponentDTO nodeReferencingComponent = nodeReferencingComponentEntity.getControllerServiceReferencingComponent();
// handle active thread counts
if (nodeReferencingComponent.getActiveThreadCount() != null && nodeReferencingComponent.getActiveThreadCount() > 0) {
final Integer current = activeThreadCounts.get(nodeReferencingComponent.getId());
if (current == null) {
activeThreadCounts.put(nodeReferencingComponent.getId(), nodeReferencingComponent.getActiveThreadCount());
} else {
activeThreadCounts.put(nodeReferencingComponent.getId(), nodeReferencingComponent.getActiveThreadCount() + current);
}
}
// handle controller service state
final String state = states.get(nodeReferencingComponent.getId());
if (state == null) {
if (ControllerServiceState.DISABLING.name().equals(nodeReferencingComponent.getState())) {
states.put(nodeReferencingComponent.getId(), ControllerServiceState.DISABLING.name());
} else if (ControllerServiceState.ENABLING.name().equals(nodeReferencingComponent.getState())) {
states.put(nodeReferencingComponent.getId(), ControllerServiceState.ENABLING.name());
}
}
}
}
}
// go through each referencing components
for (final ControllerServiceReferencingComponentEntity referencingComponent : referencingComponents) {
final Integer activeThreadCount = activeThreadCounts.get(referencingComponent.getId());
if (activeThreadCount != null) {
referencingComponent.getControllerServiceReferencingComponent().setActiveThreadCount(activeThreadCount);
}
final String state = states.get(referencingComponent.getId());
if (state != null) {
referencingComponent.getControllerServiceReferencingComponent().setState(state);
}
}
}
}

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.coordination.http.endpoints;
import java.net.URI;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
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.protocol.NodeIdentifier;
import org.apache.nifi.web.api.dto.ControllerServiceReferencingComponentDTO;
import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentEntity;
import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentsEntity;
public class ControllerServiceReferenceEndpointMerger implements EndpointResponseMerger {
public static final Pattern CONTROLLER_SERVICE_REFERENCES_URI_PATTERN = Pattern.compile("/nifi-api/controller-services/node/[a-f0-9\\-]{36}/references");
@Override
public boolean canHandle(URI uri, String method) {
if (("GET".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method)) && CONTROLLER_SERVICE_REFERENCES_URI_PATTERN.matcher(uri.getPath()).matches()) {
return true;
}
return false;
}
@Override
public NodeResponse merge(URI uri, String method, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses, 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 ControllerServiceReferencingComponentsEntity responseEntity = clientResponse.getClientResponse().getEntity(ControllerServiceReferencingComponentsEntity.class);
final Set<ControllerServiceReferencingComponentEntity> referencingComponents = responseEntity.getControllerServiceReferencingComponents();
final Map<NodeIdentifier, Set<ControllerServiceReferencingComponentEntity>> resultsMap = new HashMap<>();
for (final NodeResponse nodeResponse : successfulResponses) {
final ControllerServiceReferencingComponentsEntity nodeResponseEntity = nodeResponse == clientResponse ? responseEntity
: nodeResponse.getClientResponse().getEntity(ControllerServiceReferencingComponentsEntity.class);
final Set<ControllerServiceReferencingComponentEntity> nodeReferencingComponents = nodeResponseEntity.getControllerServiceReferencingComponents();
resultsMap.put(nodeResponse.getNodeId(), nodeReferencingComponents);
}
ControllerServiceEndpointMerger.mergeControllerServiceReferences(referencingComponents, resultsMap);
return new NodeResponse(clientResponse, responseEntity);
}
}

View File

@ -0,0 +1,56 @@
/*
* 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.Map;
import java.util.Set;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.web.api.dto.ControllerServiceDTO;
import org.apache.nifi.web.api.entity.ControllerServicesEntity;
public class ControllerServicesEndpointMerger extends AbstractMultiEntityEndpoint<ControllerServicesEntity, ControllerServiceDTO> {
public static final String CONTROLLER_SERVICES_URI = "/nifi-api/controller-services/node";
@Override
public boolean canHandle(URI uri, String method) {
return "GET".equalsIgnoreCase(method) && CONTROLLER_SERVICES_URI.equals(uri.getPath());
}
@Override
protected Class<ControllerServicesEntity> getEntityClass() {
return ControllerServicesEntity.class;
}
@Override
protected Set<ControllerServiceDTO> getDtos(ControllerServicesEntity entity) {
return entity.getControllerServices();
}
@Override
protected String getComponentId(ControllerServiceDTO dto) {
return dto.getId();
}
@Override
protected void mergeResponses(ControllerServiceDTO clientDto, Map<NodeIdentifier, ControllerServiceDTO> dtoMap, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses) {
new ControllerServiceEndpointMerger().mergeResponses(clientDto, dtoMap, successfulResponses, problematicResponses);
}
}

View File

@ -0,0 +1,83 @@
/*
* 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.Map;
import java.util.Set;
import java.util.regex.Pattern;
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.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 static final Pattern CONTROLLER_STATUS_URI_PATTERN = Pattern.compile("/nifi-api/flow/status");
@Override
public boolean canHandle(URI uri, String method) {
return "GET".equalsIgnoreCase(method) && CONTROLLER_STATUS_URI_PATTERN.matcher(uri.getPath()).matches();
}
@Override
protected Class<ControllerStatusEntity> getEntityClass() {
return ControllerStatusEntity.class;
}
@Override
protected ControllerStatusDTO getDto(ControllerStatusEntity entity) {
return entity.getControllerStatus();
}
@Override
protected void mergeResponses(ControllerStatusDTO clientDto, Map<NodeIdentifier, ControllerStatusDTO> dtoMap, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses) {
ControllerStatusDTO mergedStatus = clientDto;
for (final Map.Entry<NodeIdentifier, ControllerStatusDTO> entry : dtoMap.entrySet()) {
final NodeIdentifier nodeId = entry.getKey();
final ControllerStatusDTO nodeStatus = entry.getValue();
final String nodeAddress = nodeId.getApiAddress() + ":" + nodeId.getApiPort();
for (final BulletinDTO bulletin : nodeStatus.getBulletins()) {
bulletin.setNodeAddress(nodeAddress);
}
for (final BulletinDTO bulletin : nodeStatus.getControllerServiceBulletins()) {
bulletin.setNodeAddress(nodeAddress);
}
for (final BulletinDTO bulletin : nodeStatus.getReportingTaskBulletins()) {
bulletin.setNodeAddress(nodeAddress);
}
if (nodeStatus == mergedStatus) {
continue;
}
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

@ -0,0 +1,73 @@
/*
* 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.ArrayList;
import java.util.Map;
import java.util.regex.Pattern;
import org.apache.nifi.cluster.manager.StatusMerger;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.web.api.dto.CountersDTO;
import org.apache.nifi.web.api.dto.NodeCountersSnapshotDTO;
import org.apache.nifi.web.api.entity.CountersEntity;
public class CountersEndpointMerger extends AbstractNodeStatusEndpoint<CountersEntity, CountersDTO> {
public static final Pattern COUNTERS_URI_PATTERN = Pattern.compile("/nifi-api/controller/counters");
@Override
public boolean canHandle(URI uri, String method) {
return "GET".equalsIgnoreCase(method) && COUNTERS_URI_PATTERN.matcher(uri.getPath()).matches();
}
@Override
protected Class<CountersEntity> getEntityClass() {
return CountersEntity.class;
}
@Override
protected CountersDTO getDto(CountersEntity entity) {
return entity.getCounters();
}
@Override
protected void mergeResponses(CountersDTO clientDto, Map<NodeIdentifier, CountersDTO> dtoMap, NodeIdentifier selectedNodeId) {
final CountersDTO mergedCounters = clientDto;
mergedCounters.setNodeSnapshots(new ArrayList<NodeCountersSnapshotDTO>());
final NodeCountersSnapshotDTO selectedNodeSnapshot = new NodeCountersSnapshotDTO();
selectedNodeSnapshot.setSnapshot(clientDto.getAggregateSnapshot().clone());
selectedNodeSnapshot.setAddress(selectedNodeId.getApiAddress());
selectedNodeSnapshot.setApiPort(selectedNodeId.getApiPort());
selectedNodeSnapshot.setNodeId(selectedNodeId.getId());
mergedCounters.getNodeSnapshots().add(selectedNodeSnapshot);
for (final Map.Entry<NodeIdentifier, CountersDTO> entry : dtoMap.entrySet()) {
final NodeIdentifier nodeId = entry.getKey();
final CountersDTO toMerge = entry.getValue();
if (toMerge == clientDto) {
continue;
}
StatusMerger.merge(mergedCounters, toMerge, nodeId.getId(), nodeId.getApiAddress(), nodeId.getApiPort());
}
}
}

View File

@ -0,0 +1,125 @@
/*
* 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.Map;
import java.util.Set;
import java.util.regex.Pattern;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.controller.queue.DropFlowFileState;
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 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}");
@Override
public boolean canHandle(URI uri, String method) {
if (("GET".equalsIgnoreCase(method) || "DELETE".equalsIgnoreCase(method)) && DROP_REQUEST_URI.matcher(uri.getPath()).matches()) {
return true;
} else if (("POST".equalsIgnoreCase(method) && DROP_REQUESTS_URI.matcher(uri.getPath()).matches())) {
return true;
}
return false;
}
@Override
protected Class<DropRequestEntity> getEntityClass() {
return DropRequestEntity.class;
}
@Override
protected DropRequestDTO getDto(DropRequestEntity entity) {
return entity.getDropRequest();
}
@Override
protected void mergeResponses(DropRequestDTO clientDto, Map<NodeIdentifier, DropRequestDTO> dtoMap, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses) {
boolean nodeWaiting = false;
int originalCount = 0;
long originalSize = 0;
int currentCount = 0;
long currentSize = 0;
int droppedCount = 0;
long droppedSize = 0;
DropFlowFileState state = null;
boolean allFinished = true;
String failureReason = null;
for (final Map.Entry<NodeIdentifier, DropRequestDTO> nodeEntry : dtoMap.entrySet()) {
final DropRequestDTO nodeDropRequest = nodeEntry.getValue();
if (!nodeDropRequest.isFinished()) {
allFinished = false;
}
if (nodeDropRequest.getFailureReason() != null) {
failureReason = nodeDropRequest.getFailureReason();
}
currentCount += nodeDropRequest.getCurrentCount();
currentSize += nodeDropRequest.getCurrentSize();
droppedCount += nodeDropRequest.getDroppedCount();
droppedSize += nodeDropRequest.getDroppedSize();
if (nodeDropRequest.getOriginalCount() == null) {
nodeWaiting = true;
} else {
originalCount += nodeDropRequest.getOriginalCount();
originalSize += nodeDropRequest.getOriginalSize();
}
final DropFlowFileState nodeState = DropFlowFileState.valueOfDescription(nodeDropRequest.getState());
if (state == null || state.ordinal() > nodeState.ordinal()) {
state = nodeState;
}
}
clientDto.setCurrentCount(currentCount);
clientDto.setCurrentSize(currentSize);
clientDto.setCurrent(FormatUtils.formatCount(currentCount) + " / " + FormatUtils.formatDataSize(currentSize));
clientDto.setDroppedCount(droppedCount);
clientDto.setDroppedSize(droppedSize);
clientDto.setDropped(FormatUtils.formatCount(droppedCount) + " / " + FormatUtils.formatDataSize(droppedSize));
clientDto.setFinished(allFinished);
clientDto.setFailureReason(failureReason);
if (originalCount == 0) {
clientDto.setPercentCompleted(allFinished ? 100 : 0);
} else {
clientDto.setPercentCompleted((int) ((double) droppedCount / (double) originalCount * 100D));
}
if (!nodeWaiting) {
clientDto.setOriginalCount(originalCount);
clientDto.setOriginalSize(originalSize);
clientDto.setOriginal(FormatUtils.formatCount(originalCount) + " / " + FormatUtils.formatDataSize(originalSize));
}
if (state != null) {
clientDto.setState(state.toString());
}
}
}

View File

@ -0,0 +1,103 @@
/*
* 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.HashMap;
import java.util.Map;
import java.util.Set;
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.protocol.NodeIdentifier;
import org.apache.nifi.web.api.dto.FlowSnippetDTO;
import org.apache.nifi.web.api.dto.ProcessorDTO;
import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
import org.apache.nifi.web.api.entity.FlowSnippetEntity;
public class FlowSnippetEndpointMerger implements EndpointResponseMerger {
public static final Pattern TEMPLATE_INSTANCE_URI_PATTERN = Pattern.compile("/nifi-api/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/template-instance");
public static final Pattern FLOW_SNIPPET_INSTANCE_URI_PATTERN = Pattern.compile("/nifi-api/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/snippet-instance");
@Override
public boolean canHandle(final URI uri, final String method) {
return "POST".equalsIgnoreCase(method) && (TEMPLATE_INSTANCE_URI_PATTERN.matcher(uri.getPath()).matches()
|| FLOW_SNIPPET_INSTANCE_URI_PATTERN.matcher(uri.getPath()).matches());
}
@Override
public NodeResponse merge(final URI uri, final String method, Set<NodeResponse> successfulResponses, final Set<NodeResponse> problematicResponses, final NodeResponse clientResponse) {
final FlowSnippetEntity responseEntity = clientResponse.getClientResponse().getEntity(FlowSnippetEntity.class);
final FlowSnippetDTO contents = responseEntity.getContents();
if (contents == null) {
return clientResponse;
} else {
final Map<String, Map<NodeIdentifier, ProcessorDTO>> processorMap = new HashMap<>();
final Map<String, Map<NodeIdentifier, RemoteProcessGroupDTO>> remoteProcessGroupMap = new HashMap<>();
for (final NodeResponse nodeResponse : successfulResponses) {
final FlowSnippetEntity nodeResponseEntity = nodeResponse == clientResponse ? responseEntity : nodeResponse.getClientResponse().getEntity(FlowSnippetEntity.class);
final FlowSnippetDTO nodeContents = nodeResponseEntity.getContents();
for (final ProcessorDTO nodeProcessor : nodeContents.getProcessors()) {
Map<NodeIdentifier, ProcessorDTO> innerMap = processorMap.get(nodeProcessor.getId());
if (innerMap == null) {
innerMap = new HashMap<>();
processorMap.put(nodeProcessor.getId(), innerMap);
}
innerMap.put(nodeResponse.getNodeId(), nodeProcessor);
}
for (final RemoteProcessGroupDTO nodeRemoteProcessGroup : nodeContents.getRemoteProcessGroups()) {
Map<NodeIdentifier, RemoteProcessGroupDTO> innerMap = remoteProcessGroupMap.get(nodeRemoteProcessGroup.getId());
if (innerMap == null) {
innerMap = new HashMap<>();
remoteProcessGroupMap.put(nodeRemoteProcessGroup.getId(), innerMap);
}
innerMap.put(nodeResponse.getNodeId(), nodeRemoteProcessGroup);
}
}
final ProcessorEndpointMerger procMerger = new ProcessorEndpointMerger();
for (final ProcessorDTO processor : contents.getProcessors()) {
final String procId = processor.getId();
final Map<NodeIdentifier, ProcessorDTO> mergeMap = processorMap.get(procId);
procMerger.mergeResponses(processor, mergeMap, successfulResponses, problematicResponses);
}
final RemoteProcessGroupEndpointMerger rpgMerger = new RemoteProcessGroupEndpointMerger();
for (final RemoteProcessGroupDTO remoteProcessGroup : contents.getRemoteProcessGroups()) {
if (remoteProcessGroup.getContents() != null) {
final String remoteProcessGroupId = remoteProcessGroup.getId();
final Map<NodeIdentifier, RemoteProcessGroupDTO> mergeMap = remoteProcessGroupMap.get(remoteProcessGroupId);
rpgMerger.mergeResponses(remoteProcessGroup, mergeMap, successfulResponses, problematicResponses);
}
}
}
// create a new client response
return new NodeResponse(clientResponse, responseEntity);
}
}

View File

@ -0,0 +1,89 @@
/*
* 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.ArrayList;
import java.util.List;
import java.util.ListIterator;
import java.util.Map;
import java.util.regex.Pattern;
import org.apache.nifi.cluster.manager.StatusMerger;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.web.api.dto.status.NodeProcessGroupStatusSnapshotDTO;
import org.apache.nifi.web.api.dto.status.ProcessGroupStatusDTO;
import org.apache.nifi.web.api.dto.status.ProcessGroupStatusSnapshotDTO;
import org.apache.nifi.web.api.dto.status.RemoteProcessGroupStatusSnapshotDTO;
import org.apache.nifi.web.api.entity.ProcessGroupStatusEntity;
public class GroupStatusEndpointMerger extends AbstractNodeStatusEndpoint<ProcessGroupStatusEntity, ProcessGroupStatusDTO> {
public static final Pattern GROUP_STATUS_URI_PATTERN = Pattern.compile("/nifi-api/flow/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/status");
@Override
public boolean canHandle(URI uri, String method) {
return "GET".equalsIgnoreCase(method) && GROUP_STATUS_URI_PATTERN.matcher(uri.getPath()).matches();
}
@Override
protected Class<ProcessGroupStatusEntity> getEntityClass() {
return ProcessGroupStatusEntity.class;
}
@Override
protected ProcessGroupStatusDTO getDto(ProcessGroupStatusEntity entity) {
return entity.getProcessGroupStatus();
}
@Override
protected void mergeResponses(ProcessGroupStatusDTO clientDto, Map<NodeIdentifier, ProcessGroupStatusDTO> dtoMap, NodeIdentifier selectedNodeId) {
final ProcessGroupStatusDTO mergedProcessGroupStatus = clientDto;
mergedProcessGroupStatus.setNodeSnapshots(new ArrayList<NodeProcessGroupStatusSnapshotDTO>());
final NodeProcessGroupStatusSnapshotDTO selectedNodeSnapshot = new NodeProcessGroupStatusSnapshotDTO();
selectedNodeSnapshot.setStatusSnapshot(clientDto.getAggregateSnapshot().clone());
selectedNodeSnapshot.setAddress(selectedNodeId.getApiAddress());
selectedNodeSnapshot.setApiPort(selectedNodeId.getApiPort());
selectedNodeSnapshot.setNodeId(selectedNodeId.getId());
mergedProcessGroupStatus.getNodeSnapshots().add(selectedNodeSnapshot);
for (final Map.Entry<NodeIdentifier, ProcessGroupStatusDTO> entry : dtoMap.entrySet()) {
final NodeIdentifier nodeId = entry.getKey();
final ProcessGroupStatusDTO nodeProcessGroupStatus = entry.getValue();
if (nodeProcessGroupStatus == mergedProcessGroupStatus) {
continue;
}
final ProcessGroupStatusSnapshotDTO nodeSnapshot = nodeProcessGroupStatus.getAggregateSnapshot();
for (final RemoteProcessGroupStatusSnapshotDTO remoteProcessGroupStatus : nodeSnapshot.getRemoteProcessGroupStatusSnapshots()) {
final List<String> nodeAuthorizationIssues = remoteProcessGroupStatus.getAuthorizationIssues();
if (!nodeAuthorizationIssues.isEmpty()) {
for (final ListIterator<String> iter = nodeAuthorizationIssues.listIterator(); iter.hasNext();) {
final String Issue = iter.next();
iter.set("[" + nodeId.getApiAddress() + ":" + nodeId.getApiPort() + "] -- " + Issue);
}
remoteProcessGroupStatus.setAuthorizationIssues(nodeAuthorizationIssues);
}
}
StatusMerger.merge(mergedProcessGroupStatus, nodeProcessGroupStatus, nodeId.getId(), nodeId.getApiAddress(), nodeId.getApiPort());
}
}
}

View File

@ -0,0 +1,156 @@
/*
* 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.ArrayList;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.NavigableSet;
import java.util.Set;
import java.util.TreeSet;
import java.util.regex.Pattern;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.controller.queue.ListFlowFileState;
import org.apache.nifi.web.api.dto.FlowFileSummaryDTO;
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 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}");
@Override
public boolean canHandle(URI uri, String method) {
if (("GET".equalsIgnoreCase(method) || "DELETE".equalsIgnoreCase(method)) && LISTING_REQUEST_URI.matcher(uri.getPath()).matches()) {
return true;
} else if ("POST".equalsIgnoreCase(method) && LISTING_REQUESTS_URI.matcher(uri.getPath()).matches()) {
return true;
}
return false;
}
@Override
protected Class<ListingRequestEntity> getEntityClass() {
return ListingRequestEntity.class;
}
@Override
protected ListingRequestDTO getDto(ListingRequestEntity entity) {
return entity.getListingRequest();
}
@Override
protected void mergeResponses(ListingRequestDTO clientDto, Map<NodeIdentifier, ListingRequestDTO> dtoMap, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses) {
final Comparator<FlowFileSummaryDTO> comparator = new Comparator<FlowFileSummaryDTO>() {
@Override
public int compare(final FlowFileSummaryDTO dto1, final FlowFileSummaryDTO dto2) {
int positionCompare = dto1.getPosition().compareTo(dto2.getPosition());
if (positionCompare != 0) {
return positionCompare;
}
final String address1 = dto1.getClusterNodeAddress();
final String address2 = dto2.getClusterNodeAddress();
if (address1 == null && address2 == null) {
return 0;
}
if (address1 == null) {
return 1;
}
if (address2 == null) {
return -1;
}
return address1.compareTo(address2);
}
};
final NavigableSet<FlowFileSummaryDTO> flowFileSummaries = new TreeSet<>(comparator);
ListFlowFileState state = null;
int numStepsCompleted = 0;
int numStepsTotal = 0;
int objectCount = 0;
long byteCount = 0;
boolean finished = true;
for (final Map.Entry<NodeIdentifier, ListingRequestDTO> entry : dtoMap.entrySet()) {
final NodeIdentifier nodeIdentifier = entry.getKey();
final String nodeAddress = nodeIdentifier.getApiAddress() + ":" + nodeIdentifier.getApiPort();
final ListingRequestDTO nodeRequest = entry.getValue();
numStepsTotal++;
if (Boolean.TRUE.equals(nodeRequest.getFinished())) {
numStepsCompleted++;
}
final QueueSizeDTO nodeQueueSize = nodeRequest.getQueueSize();
objectCount += nodeQueueSize.getObjectCount();
byteCount += nodeQueueSize.getByteCount();
if (!nodeRequest.getFinished()) {
finished = false;
}
if (nodeRequest.getLastUpdated().after(clientDto.getLastUpdated())) {
clientDto.setLastUpdated(nodeRequest.getLastUpdated());
}
// Keep the state with the lowest ordinal value (the "least completed").
final ListFlowFileState nodeState = ListFlowFileState.valueOfDescription(nodeRequest.getState());
if (state == null || state.compareTo(nodeState) > 0) {
state = nodeState;
}
if (nodeRequest.getFlowFileSummaries() != null) {
for (final FlowFileSummaryDTO summaryDTO : nodeRequest.getFlowFileSummaries()) {
summaryDTO.setClusterNodeId(nodeIdentifier.getId());
summaryDTO.setClusterNodeAddress(nodeAddress);
flowFileSummaries.add(summaryDTO);
// Keep the set from growing beyond our max
if (flowFileSummaries.size() > clientDto.getMaxResults()) {
flowFileSummaries.pollLast();
}
}
}
if (nodeRequest.getFailureReason() != null) {
clientDto.setFailureReason(nodeRequest.getFailureReason());
}
}
final List<FlowFileSummaryDTO> summaryDTOs = new ArrayList<>(flowFileSummaries);
clientDto.setFlowFileSummaries(summaryDTOs);
// depends on invariant if numStepsTotal is 0, so is numStepsCompleted, all steps being completed
// would be 1
final int percentCompleted = (numStepsTotal == 0) ? 1 : numStepsCompleted / numStepsTotal;
clientDto.setPercentCompleted(percentCompleted);
clientDto.setFinished(finished);
clientDto.getQueueSize().setByteCount(byteCount);
clientDto.getQueueSize().setObjectCount(objectCount);
}
}

View File

@ -0,0 +1,75 @@
/*
* 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.ArrayList;
import java.util.Map;
import java.util.regex.Pattern;
import org.apache.nifi.cluster.manager.StatusMerger;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.web.api.dto.status.NodePortStatusSnapshotDTO;
import org.apache.nifi.web.api.dto.status.PortStatusDTO;
import org.apache.nifi.web.api.entity.PortStatusEntity;
public class PortStatusEndpointMerger extends AbstractNodeStatusEndpoint<PortStatusEntity, PortStatusDTO> {
public static final Pattern INPUT_PORT_STATUS_URI_PATTERN = Pattern.compile("/nifi-api/flow/input-ports/[a-f0-9\\-]{36}/status");
public static final Pattern OUTPUT_PORT_STATUS_URI_PATTERN = Pattern.compile("/nifi-api/flow/output-ports/[a-f0-9\\-]{36}/status");
@Override
public boolean canHandle(URI uri, String method) {
return "GET".equalsIgnoreCase(method) && (INPUT_PORT_STATUS_URI_PATTERN.matcher(uri.getPath()).matches() || OUTPUT_PORT_STATUS_URI_PATTERN.matcher(uri.getPath()).matches());
}
@Override
protected Class<PortStatusEntity> getEntityClass() {
return PortStatusEntity.class;
}
@Override
protected PortStatusDTO getDto(PortStatusEntity entity) {
return entity.getPortStatus();
}
@Override
protected void mergeResponses(PortStatusDTO clientDto, Map<NodeIdentifier, PortStatusDTO> dtoMap, NodeIdentifier selectedNodeId) {
final PortStatusDTO mergedPortStatus = clientDto;
mergedPortStatus.setNodeSnapshots(new ArrayList<NodePortStatusSnapshotDTO>());
final NodePortStatusSnapshotDTO selectedNodeSnapshot = new NodePortStatusSnapshotDTO();
selectedNodeSnapshot.setStatusSnapshot(clientDto.getAggregateSnapshot().clone());
selectedNodeSnapshot.setAddress(selectedNodeId.getApiAddress());
selectedNodeSnapshot.setApiPort(selectedNodeId.getApiPort());
selectedNodeSnapshot.setNodeId(selectedNodeId.getId());
mergedPortStatus.getNodeSnapshots().add(selectedNodeSnapshot);
// merge the other nodes
for (final Map.Entry<NodeIdentifier, PortStatusDTO> entry : dtoMap.entrySet()) {
final NodeIdentifier nodeId = entry.getKey();
final PortStatusDTO nodePortStatus = entry.getValue();
if (nodePortStatus == clientDto) {
continue;
}
StatusMerger.merge(mergedPortStatus, nodePortStatus, nodeId.getId(), nodeId.getApiAddress(), nodeId.getApiPort());
}
}
}

View File

@ -0,0 +1,107 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.coordination.http.endpoints;
import java.net.URI;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
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.protocol.NodeIdentifier;
import org.apache.nifi.web.api.dto.FlowSnippetDTO;
import org.apache.nifi.web.api.dto.ProcessGroupDTO;
import org.apache.nifi.web.api.dto.ProcessorDTO;
import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
import org.apache.nifi.web.api.entity.ProcessGroupEntity;
public class ProcessGroupEndpointMerger implements EndpointResponseMerger {
public static final Pattern PROCESS_GROUP_URI_PATTERN = Pattern.compile("/nifi-api/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))");
@Override
public boolean canHandle(final URI uri, final String method) {
return ("GET".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method)) && PROCESS_GROUP_URI_PATTERN.matcher(uri.getPath()).matches();
}
@Override
public 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 ProcessGroupEntity responseEntity = clientResponse.getClientResponse().getEntity(ProcessGroupEntity.class);
final ProcessGroupDTO responseDto = responseEntity.getComponent();
final FlowSnippetDTO contents = responseDto.getContents();
if (contents == null) {
return new NodeResponse(clientResponse, responseEntity);
} else {
final Map<String, Map<NodeIdentifier, ProcessorDTO>> processorMap = new HashMap<>();
final Map<String, Map<NodeIdentifier, RemoteProcessGroupDTO>> remoteProcessGroupMap = new HashMap<>();
for (final NodeResponse nodeResponse : successfulResponses) {
final ProcessGroupEntity nodeResponseEntity = nodeResponse == clientResponse ? responseEntity : nodeResponse.getClientResponse().getEntity(ProcessGroupEntity.class);
final ProcessGroupDTO nodeProcessGroup = nodeResponseEntity.getComponent();
for (final ProcessorDTO nodeProcessor : nodeProcessGroup.getContents().getProcessors()) {
Map<NodeIdentifier, ProcessorDTO> innerMap = processorMap.get(nodeProcessor.getId());
if (innerMap == null) {
innerMap = new HashMap<>();
processorMap.put(nodeProcessor.getId(), innerMap);
}
innerMap.put(nodeResponse.getNodeId(), nodeProcessor);
}
for (final RemoteProcessGroupDTO nodeRemoteProcessGroup : nodeProcessGroup.getContents().getRemoteProcessGroups()) {
Map<NodeIdentifier, RemoteProcessGroupDTO> innerMap = remoteProcessGroupMap.get(nodeRemoteProcessGroup.getId());
if (innerMap == null) {
innerMap = new HashMap<>();
remoteProcessGroupMap.put(nodeRemoteProcessGroup.getId(), innerMap);
}
innerMap.put(nodeResponse.getNodeId(), nodeRemoteProcessGroup);
}
}
final ProcessorEndpointMerger procMerger = new ProcessorEndpointMerger();
for (final ProcessorDTO processor : contents.getProcessors()) {
final String procId = processor.getId();
final Map<NodeIdentifier, ProcessorDTO> mergeMap = processorMap.get(procId);
procMerger.mergeResponses(processor, mergeMap, successfulResponses, problematicResponses);
}
final RemoteProcessGroupEndpointMerger rpgMerger = new RemoteProcessGroupEndpointMerger();
for (final RemoteProcessGroupDTO remoteProcessGroup : contents.getRemoteProcessGroups()) {
if (remoteProcessGroup.getContents() != null) {
final String remoteProcessGroupId = remoteProcessGroup.getId();
final Map<NodeIdentifier, RemoteProcessGroupDTO> mergeMap = remoteProcessGroupMap.get(remoteProcessGroupId);
rpgMerger.mergeResponses(remoteProcessGroup, mergeMap, successfulResponses, problematicResponses);
}
}
}
// create a new client response
return new NodeResponse(clientResponse, responseEntity);
}
}

View File

@ -0,0 +1,75 @@
/*
* 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.HashMap;
import java.util.Map;
import java.util.Set;
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.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 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}");
@Override
public boolean canHandle(final URI uri, final String method) {
if (("GET".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method))
&& (PROCESSOR_URI_PATTERN.matcher(uri.getPath()).matches() || CLUSTER_PROCESSOR_URI_PATTERN.matcher(uri.getPath()).matches())) {
return true;
} else if ("POST".equalsIgnoreCase(method) && PROCESSORS_URI_PATTERN.matcher(uri.getPath()).matches()) {
return true;
}
return false;
}
@Override
protected Class<ProcessorEntity> getEntityClass() {
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<>();
for (final Map.Entry<NodeIdentifier, ProcessorDTO> nodeEntry : dtoMap.entrySet()) {
final NodeIdentifier nodeId = nodeEntry.getKey();
final ProcessorDTO nodeProcessor = nodeEntry.getValue();
// merge the validation errors
mergeValidationErrors(validationErrorMap, nodeId, nodeProcessor.getValidationErrors());
}
// set the merged the validation errors
clientDto.setValidationErrors(normalizedMergedValidationErrors(validationErrorMap, dtoMap.size()));
}
}

View File

@ -0,0 +1,74 @@
/*
* 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.ArrayList;
import java.util.Map;
import java.util.regex.Pattern;
import org.apache.nifi.cluster.manager.StatusMerger;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.web.api.dto.status.NodeProcessorStatusSnapshotDTO;
import org.apache.nifi.web.api.dto.status.ProcessorStatusDTO;
import org.apache.nifi.web.api.entity.ProcessorStatusEntity;
public class ProcessorStatusEndpointMerger extends AbstractNodeStatusEndpoint<ProcessorStatusEntity, ProcessorStatusDTO> {
public static final Pattern PROCESSOR_STATUS_URI_PATTERN = Pattern.compile("/nifi-api/flow/processors/[a-f0-9\\-]{36}/status");
@Override
public boolean canHandle(URI uri, String method) {
return "GET".equalsIgnoreCase(method) && PROCESSOR_STATUS_URI_PATTERN.matcher(uri.getPath()).matches();
}
@Override
protected Class<ProcessorStatusEntity> getEntityClass() {
return ProcessorStatusEntity.class;
}
@Override
protected ProcessorStatusDTO getDto(ProcessorStatusEntity entity) {
return entity.getProcessorStatus();
}
@Override
protected void mergeResponses(ProcessorStatusDTO clientDto, Map<NodeIdentifier, ProcessorStatusDTO> dtoMap, NodeIdentifier selectedNodeId) {
final ProcessorStatusDTO mergedProcessorStatus = clientDto;
mergedProcessorStatus.setNodeSnapshots(new ArrayList<NodeProcessorStatusSnapshotDTO>());
final NodeProcessorStatusSnapshotDTO selectedNodeSnapshot = new NodeProcessorStatusSnapshotDTO();
selectedNodeSnapshot.setStatusSnapshot(clientDto.getAggregateSnapshot().clone());
selectedNodeSnapshot.setAddress(selectedNodeId.getApiAddress());
selectedNodeSnapshot.setApiPort(selectedNodeId.getApiPort());
selectedNodeSnapshot.setNodeId(selectedNodeId.getId());
mergedProcessorStatus.getNodeSnapshots().add(selectedNodeSnapshot);
// merge the other nodes
for (final Map.Entry<NodeIdentifier, ProcessorStatusDTO> entry : dtoMap.entrySet()) {
final NodeIdentifier nodeId = entry.getKey();
final ProcessorStatusDTO nodeProcessorStatus = entry.getValue();
if (nodeProcessorStatus == clientDto) {
continue;
}
StatusMerger.merge(mergedProcessorStatus, nodeProcessorStatus, nodeId.getId(), nodeId.getApiAddress(), nodeId.getApiPort());
}
}
}

View File

@ -0,0 +1,78 @@
/*
* 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.HashMap;
import java.util.Map;
import java.util.Set;
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.protocol.NodeIdentifier;
import org.apache.nifi.web.api.dto.ProcessorDTO;
import org.apache.nifi.web.api.entity.ProcessorEntity;
import org.apache.nifi.web.api.entity.ProcessorsEntity;
public class ProcessorsEndpointMerger implements EndpointResponseMerger {
public static final Pattern PROCESSORS_URI_PATTERN = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/processors");
@Override
public boolean canHandle(final URI uri, final String method) {
return "GET".equalsIgnoreCase(method) && PROCESSORS_URI_PATTERN.matcher(uri.getPath()).matches();
}
@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 ProcessorsEntity responseEntity = clientResponse.getClientResponse().getEntity(ProcessorsEntity.class);
final Set<ProcessorEntity> processorEntities = responseEntity.getProcessors();
final Map<String, Map<NodeIdentifier, ProcessorDTO>> dtoMap = new HashMap<>();
for (final NodeResponse nodeResponse : successfulResponses) {
final ProcessorsEntity nodeResponseEntity = nodeResponse == clientResponse ? responseEntity : nodeResponse.getClientResponse().getEntity(ProcessorsEntity.class);
final Set<ProcessorEntity> nodeProcessorEntities = nodeResponseEntity.getProcessors();
for (final ProcessorEntity nodeProcessorEntity : nodeProcessorEntities) {
final NodeIdentifier nodeId = nodeResponse.getNodeId();
Map<NodeIdentifier, ProcessorDTO> innerMap = dtoMap.get(nodeId);
if (innerMap == null) {
innerMap = new HashMap<>();
dtoMap.put(nodeProcessorEntity.getId(), innerMap);
}
innerMap.put(nodeResponse.getNodeId(), nodeProcessorEntity.getComponent());
}
}
final ProcessorEndpointMerger procMerger = new ProcessorEndpointMerger();
for (final ProcessorEntity entity : processorEntities) {
final String componentId = entity.getId();
final Map<NodeIdentifier, ProcessorDTO> mergeMap = dtoMap.get(componentId);
procMerger.mergeResponses(entity.getComponent(), mergeMap, successfulResponses, problematicResponses);
}
// create a new client response
return new NodeResponse(clientResponse, responseEntity);
}
}

View File

@ -0,0 +1,55 @@
/*
* 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.Map;
import java.util.Set;
import java.util.regex.Pattern;
import org.apache.nifi.cluster.manager.NodeResponse;
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 static final Pattern PROVENANCE_EVENT_URI = Pattern.compile("/nifi-api/provenance/events/[0-9]+");
@Override
public boolean canHandle(URI uri, String method) {
return "GET".equalsIgnoreCase(method) && PROVENANCE_EVENT_URI.matcher(uri.getPath()).matches();
}
@Override
protected Class<ProvenanceEventEntity> getEntityClass() {
return ProvenanceEventEntity.class;
}
@Override
protected ProvenanceEventDTO getDto(ProvenanceEventEntity entity) {
return entity.getProvenanceEvent();
}
@Override
protected void mergeResponses(ProvenanceEventDTO clientDto, Map<NodeIdentifier, ProvenanceEventDTO> dtoMap, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses) {
// The request for a Provenance Event is replicated to a single Node. We simply update its cluster node info.
final NodeIdentifier nodeId = successfulResponses.iterator().next().getNodeId();
clientDto.setClusterNodeId(nodeId.getId());
clientDto.setClusterNodeAddress(nodeId.getApiAddress() + ":" + nodeId.getApiPort());
}
}

View File

@ -0,0 +1,187 @@
/*
* 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.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.Date;
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;
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.util.FormatUtils;
import org.apache.nifi.web.api.dto.provenance.ProvenanceDTO;
import org.apache.nifi.web.api.dto.provenance.ProvenanceEventDTO;
import org.apache.nifi.web.api.dto.provenance.ProvenanceRequestDTO;
import org.apache.nifi.web.api.dto.provenance.ProvenanceResultsDTO;
import org.apache.nifi.web.api.entity.ProvenanceEntity;
public class ProvenanceQueryEndpointMerger implements EndpointResponseMerger {
public static final String PROVENANCE_URI = "/nifi-api/provenance";
public static final Pattern PROVENANCE_QUERY_URI = Pattern.compile("/nifi-api/provenance/[a-f0-9\\-]{36}");
@Override
public boolean canHandle(URI uri, String method) {
if ("POST".equalsIgnoreCase(method) && PROVENANCE_URI.equals(uri.getPath())) {
return true;
} else if ("GET".equalsIgnoreCase(method) && PROVENANCE_QUERY_URI.matcher(uri.getPath()).matches()) {
return true;
}
return false;
}
@Override
public NodeResponse merge(URI uri, String method, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses, 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 ProvenanceEntity responseEntity = clientResponse.getClientResponse().getEntity(ProvenanceEntity.class);
final ProvenanceDTO dto = responseEntity.getProvenance();
final Map<NodeIdentifier, ProvenanceDTO> dtoMap = new HashMap<>();
for (final NodeResponse nodeResponse : successfulResponses) {
final ProvenanceEntity nodeResponseEntity = nodeResponse == clientResponse ? responseEntity : nodeResponse.getClientResponse().getEntity(ProvenanceEntity.class);
final ProvenanceDTO nodeDto = nodeResponseEntity.getProvenance();
dtoMap.put(nodeResponse.getNodeId(), nodeDto);
}
mergeResponses(dto, dtoMap, successfulResponses, problematicResponses);
return new NodeResponse(clientResponse, responseEntity);
}
protected void mergeResponses(ProvenanceDTO clientDto, Map<NodeIdentifier, ProvenanceDTO> dtoMap, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses) {
final ProvenanceResultsDTO results = clientDto.getResults();
final ProvenanceRequestDTO request = clientDto.getRequest();
final List<ProvenanceEventDTO> allResults = new ArrayList<>(1024);
final Set<String> errors = new HashSet<>();
Date oldestEventDate = new Date();
int percentageComplete = 0;
boolean finished = true;
long totalRecords = 0;
for (final Map.Entry<NodeIdentifier, ProvenanceDTO> entry : dtoMap.entrySet()) {
final NodeIdentifier nodeIdentifier = entry.getKey();
final String nodeAddress = nodeIdentifier.getApiAddress() + ":" + nodeIdentifier.getApiPort();
final ProvenanceDTO nodeDto = entry.getValue();
final ProvenanceResultsDTO nodeResultDto = nodeDto.getResults();
if (nodeResultDto != null && nodeResultDto.getProvenanceEvents() != null) {
// increment the total number of records
totalRecords += nodeResultDto.getTotalCount();
// populate the cluster identifier
for (final ProvenanceEventDTO eventDto : nodeResultDto.getProvenanceEvents()) {
eventDto.setClusterNodeId(nodeIdentifier.getId());
eventDto.setClusterNodeAddress(nodeAddress);
// add node identifier to the event's id so that it is unique across cluster
eventDto.setId(nodeIdentifier.getId() + eventDto.getId());
allResults.add(eventDto);
}
}
if (nodeResultDto.getOldestEvent() != null && nodeResultDto.getOldestEvent().before(oldestEventDate)) {
oldestEventDate = nodeResultDto.getOldestEvent();
}
if (nodeResultDto.getErrors() != null) {
for (final String error : nodeResultDto.getErrors()) {
errors.add(nodeAddress + " -- " + error);
}
}
percentageComplete += nodeDto.getPercentCompleted();
if (!nodeDto.isFinished()) {
finished = false;
}
}
percentageComplete /= dtoMap.size();
// consider any problematic responses as errors
for (final NodeResponse problematicResponse : problematicResponses) {
final NodeIdentifier problemNode = problematicResponse.getNodeId();
final String problemNodeAddress = problemNode.getApiAddress() + ":" + problemNode.getApiPort();
errors.add(String.format("%s -- Request did not complete successfully (Status code: %s)", problemNodeAddress, problematicResponse.getStatus()));
}
// Since we get back up to the maximum number of results from each node, we need to sort those values and then
// grab only the first X number of them. We do a sort based on time, such that the newest are included.
// If 2 events have the same timestamp, we do a secondary sort based on Cluster Node Identifier. If those are
// equal, we perform a terciary sort based on the the event id
Collections.sort(allResults, new Comparator<ProvenanceEventDTO>() {
@Override
public int compare(final ProvenanceEventDTO o1, final ProvenanceEventDTO o2) {
final int eventTimeComparison = o1.getEventTime().compareTo(o2.getEventTime());
if (eventTimeComparison != 0) {
return -eventTimeComparison;
}
final String nodeId1 = o1.getClusterNodeId();
final String nodeId2 = o2.getClusterNodeId();
final int nodeIdComparison;
if (nodeId1 == null && nodeId2 == null) {
nodeIdComparison = 0;
} else if (nodeId1 == null) {
nodeIdComparison = 1;
} else if (nodeId2 == null) {
nodeIdComparison = -1;
} else {
nodeIdComparison = -nodeId1.compareTo(nodeId2);
}
if (nodeIdComparison != 0) {
return nodeIdComparison;
}
return -Long.compare(o1.getEventId(), o2.getEventId());
}
});
final int maxResults = request.getMaxResults().intValue();
final List<ProvenanceEventDTO> selectedResults;
if (allResults.size() < maxResults) {
selectedResults = allResults;
} else {
selectedResults = allResults.subList(0, maxResults);
}
// include any errors
if (errors.size() > 0) {
results.setErrors(errors);
}
results.setTotalCount(totalRecords);
results.setTotal(FormatUtils.formatCount(totalRecords));
results.setProvenanceEvents(selectedResults);
results.setOldestEvent(oldestEventDate);
results.setGenerated(new Date());
clientDto.setPercentCompleted(percentageComplete);
clientDto.setFinished(finished);
}
}

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.http.endpoints;
import java.net.URI;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.regex.Pattern;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.web.api.dto.RemoteProcessGroupContentsDTO;
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 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}");
@Override
public boolean canHandle(final URI uri, final String method) {
if (("GET".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method)) && REMOTE_PROCESS_GROUP_URI_PATTERN.matcher(uri.getPath()).matches()) {
return true;
} else if ("POST".equalsIgnoreCase(method) && REMOTE_PROCESS_GROUPS_URI_PATTERN.matcher(uri.getPath()).matches()) {
return true;
}
return false;
}
@Override
protected Class<RemoteProcessGroupEntity> getEntityClass() {
return RemoteProcessGroupEntity.class;
}
@Override
protected RemoteProcessGroupDTO getDto(final RemoteProcessGroupEntity entity) {
return entity.getComponent();
}
@Override
protected void mergeResponses(RemoteProcessGroupDTO clientDto, Map<NodeIdentifier, RemoteProcessGroupDTO> dtoMap, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses) {
final RemoteProcessGroupContentsDTO remoteProcessGroupContents = clientDto.getContents();
Boolean mergedIsTargetSecure = null;
final List<String> mergedAuthorizationIssues = new ArrayList<>();
final Set<RemoteProcessGroupPortDTO> mergedInputPorts = new HashSet<>();
final Set<RemoteProcessGroupPortDTO> mergedOutputPorts = new HashSet<>();
for (final Map.Entry<NodeIdentifier, RemoteProcessGroupDTO> nodeEntry : dtoMap.entrySet()) {
final NodeIdentifier nodeId = nodeEntry.getKey();
final RemoteProcessGroupDTO nodeRemoteProcessGroupDto = nodeEntry.getValue();
// merge the issues
final List<String> nodeAuthorizationIssues = nodeRemoteProcessGroupDto.getAuthorizationIssues();
if (nodeAuthorizationIssues != null && !nodeAuthorizationIssues.isEmpty()) {
for (final String nodeAuthorizationIssue : nodeAuthorizationIssues) {
mergedAuthorizationIssues.add(nodeId.getApiAddress() + ":" + nodeId.getApiPort() + " -- " + nodeAuthorizationIssue);
}
}
// use the first target secure flag since they will all be the same
final Boolean nodeIsTargetSecure = nodeRemoteProcessGroupDto.isTargetSecure();
if (mergedIsTargetSecure == null) {
mergedIsTargetSecure = nodeIsTargetSecure;
}
// merge the ports in the contents
final RemoteProcessGroupContentsDTO nodeRemoteProcessGroupContentsDto = nodeRemoteProcessGroupDto.getContents();
if (remoteProcessGroupContents != null && nodeRemoteProcessGroupContentsDto != null) {
if (nodeRemoteProcessGroupContentsDto.getInputPorts() != null) {
mergedInputPorts.addAll(nodeRemoteProcessGroupContentsDto.getInputPorts());
}
if (nodeRemoteProcessGroupContentsDto.getOutputPorts() != null) {
mergedOutputPorts.addAll(nodeRemoteProcessGroupContentsDto.getOutputPorts());
}
}
}
if (remoteProcessGroupContents != null) {
if (!mergedInputPorts.isEmpty()) {
remoteProcessGroupContents.setInputPorts(mergedInputPorts);
}
if (!mergedOutputPorts.isEmpty()) {
remoteProcessGroupContents.setOutputPorts(mergedOutputPorts);
}
}
if (mergedIsTargetSecure != null) {
clientDto.setTargetSecure(mergedIsTargetSecure);
}
if (!mergedAuthorizationIssues.isEmpty()) {
clientDto.setAuthorizationIssues(mergedAuthorizationIssues);
}
}
}

View File

@ -0,0 +1,74 @@
/*
* 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.ArrayList;
import java.util.Map;
import java.util.regex.Pattern;
import org.apache.nifi.cluster.manager.StatusMerger;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.web.api.dto.status.NodeRemoteProcessGroupStatusSnapshotDTO;
import org.apache.nifi.web.api.dto.status.RemoteProcessGroupStatusDTO;
import org.apache.nifi.web.api.entity.RemoteProcessGroupStatusEntity;
public class RemoteProcessGroupStatusEndpointMerger extends AbstractNodeStatusEndpoint<RemoteProcessGroupStatusEntity, RemoteProcessGroupStatusDTO> {
public static final Pattern REMOTE_PROCESS_GROUP_STATUS_URI_PATTERN = Pattern.compile("/nifi-api/flow/remote-process-groups/[a-f0-9\\-]{36}/status");
@Override
public boolean canHandle(URI uri, String method) {
return "GET".equalsIgnoreCase(method) && REMOTE_PROCESS_GROUP_STATUS_URI_PATTERN.matcher(uri.getPath()).matches();
}
@Override
protected Class<RemoteProcessGroupStatusEntity> getEntityClass() {
return RemoteProcessGroupStatusEntity.class;
}
@Override
protected RemoteProcessGroupStatusDTO getDto(RemoteProcessGroupStatusEntity entity) {
return entity.getRemoteProcessGroupStatus();
}
@Override
protected void mergeResponses(RemoteProcessGroupStatusDTO clientDto, Map<NodeIdentifier, RemoteProcessGroupStatusDTO> dtoMap, NodeIdentifier selectedNodeId) {
final RemoteProcessGroupStatusDTO mergedRemoteProcessGroupStatus = clientDto;
mergedRemoteProcessGroupStatus.setNodeSnapshots(new ArrayList<NodeRemoteProcessGroupStatusSnapshotDTO>());
final NodeRemoteProcessGroupStatusSnapshotDTO selectedNodeSnapshot = new NodeRemoteProcessGroupStatusSnapshotDTO();
selectedNodeSnapshot.setStatusSnapshot(clientDto.getAggregateSnapshot().clone());
selectedNodeSnapshot.setAddress(selectedNodeId.getApiAddress());
selectedNodeSnapshot.setApiPort(selectedNodeId.getApiPort());
selectedNodeSnapshot.setNodeId(selectedNodeId.getId());
mergedRemoteProcessGroupStatus.getNodeSnapshots().add(selectedNodeSnapshot);
// merge the other nodes
for (final Map.Entry<NodeIdentifier, RemoteProcessGroupStatusDTO> entry : dtoMap.entrySet()) {
final NodeIdentifier nodeId = entry.getKey();
final RemoteProcessGroupStatusDTO nodeRemoteProcessGroupStatus = entry.getValue();
if (nodeRemoteProcessGroupStatus == clientDto) {
continue;
}
StatusMerger.merge(mergedRemoteProcessGroupStatus, nodeRemoteProcessGroupStatus, nodeId.getId(), nodeId.getApiAddress(), nodeId.getApiPort());
}
}
}

View File

@ -0,0 +1,78 @@
/*
* 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.HashMap;
import java.util.Map;
import java.util.Set;
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.protocol.NodeIdentifier;
import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
import org.apache.nifi.web.api.entity.RemoteProcessGroupsEntity;
public class RemoteProcessGroupsEndpointMerger implements EndpointResponseMerger {
public static final Pattern REMOTE_PROCESS_GROUPS_URI_PATTERN = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/remote-process-groups");
@Override
public boolean canHandle(final URI uri, final String method) {
return "GET".equalsIgnoreCase(method) && REMOTE_PROCESS_GROUPS_URI_PATTERN.matcher(uri.getPath()).matches();
}
@Override
public NodeResponse merge(URI uri, String method, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses, 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 RemoteProcessGroupsEntity responseEntity = clientResponse.getClientResponse().getEntity(RemoteProcessGroupsEntity.class);
final Set<RemoteProcessGroupEntity> rpgEntities = responseEntity.getRemoteProcessGroups();
final Map<String, Map<NodeIdentifier, RemoteProcessGroupDTO>> dtoMap = new HashMap<>();
for (final NodeResponse nodeResponse : successfulResponses) {
final RemoteProcessGroupsEntity nodeResponseEntity = nodeResponse == clientResponse ? responseEntity : nodeResponse.getClientResponse().getEntity(RemoteProcessGroupsEntity.class);
final Set<RemoteProcessGroupEntity> nodeRpgEntities = nodeResponseEntity.getRemoteProcessGroups();
for (final RemoteProcessGroupEntity nodeRpgEntity : nodeRpgEntities) {
final NodeIdentifier nodeId = nodeResponse.getNodeId();
Map<NodeIdentifier, RemoteProcessGroupDTO> innerMap = dtoMap.get(nodeId);
if (innerMap == null) {
innerMap = new HashMap<>();
dtoMap.put(nodeRpgEntity.getId(), innerMap);
}
innerMap.put(nodeResponse.getNodeId(), nodeRpgEntity.getComponent());
}
}
final RemoteProcessGroupEndpointMerger rpgMerger = new RemoteProcessGroupEndpointMerger();
for (final RemoteProcessGroupEntity entity : rpgEntities) {
final String componentId = entity.getId();
final Map<NodeIdentifier, RemoteProcessGroupDTO> mergeMap = dtoMap.get(componentId);
rpgMerger.mergeResponses(entity.getComponent(), mergeMap, successfulResponses, problematicResponses);
}
// create a new client response
return new NodeResponse(clientResponse, responseEntity);
}
}

View File

@ -0,0 +1,80 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.coordination.http.endpoints;
import java.net.URI;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import java.util.regex.Pattern;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.web.api.dto.ReportingTaskDTO;
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
public class ReportingTaskEndpointMerger extends AbstractSingleEntityEndpoint<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}");
@Override
public boolean canHandle(URI uri, String method) {
if (("GET".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method)) && REPORTING_TASK_URI_PATTERN.matcher(uri.getPath()).matches()) {
return true;
} else if ("POST".equalsIgnoreCase(method) && REPORTING_TASKS_URI.equals(uri.getPath())) {
return true;
}
return false;
}
@Override
protected Class<ReportingTaskEntity> getEntityClass() {
return ReportingTaskEntity.class;
}
@Override
protected ReportingTaskDTO getDto(ReportingTaskEntity entity) {
return entity.getReportingTask();
}
@Override
protected void mergeResponses(ReportingTaskDTO clientDto, Map<NodeIdentifier, ReportingTaskDTO> dtoMap, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses) {
final Map<String, Set<NodeIdentifier>> validationErrorMap = new HashMap<>();
int activeThreadCount = 0;
for (final Map.Entry<NodeIdentifier, ReportingTaskDTO> nodeEntry : dtoMap.entrySet()) {
final NodeIdentifier nodeId = nodeEntry.getKey();
final ReportingTaskDTO nodeReportingTask = nodeEntry.getValue();
if (nodeReportingTask.getActiveThreadCount() != null) {
activeThreadCount += nodeReportingTask.getActiveThreadCount();
}
// merge the validation errors
mergeValidationErrors(validationErrorMap, nodeId, nodeReportingTask.getValidationErrors());
}
// set the merged active thread counts
clientDto.setActiveThreadCount(activeThreadCount);
// set the merged the validation errors
clientDto.setValidationErrors(normalizedMergedValidationErrors(validationErrorMap, dtoMap.size()));
}
}

View File

@ -0,0 +1,56 @@
/*
* 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.Map;
import java.util.Set;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.web.api.dto.ReportingTaskDTO;
import org.apache.nifi.web.api.entity.ReportingTasksEntity;
public class ReportingTasksEndpointMerger extends AbstractMultiEntityEndpoint<ReportingTasksEntity, ReportingTaskDTO> {
public static final String REPORTING_TASKS_URI = "/nifi-api/controller/reporting-tasks/node";
@Override
public boolean canHandle(URI uri, String method) {
return "GET".equalsIgnoreCase(method) && REPORTING_TASKS_URI.equals(uri.getPath());
}
@Override
protected Class<ReportingTasksEntity> getEntityClass() {
return ReportingTasksEntity.class;
}
@Override
protected Set<ReportingTaskDTO> getDtos(ReportingTasksEntity entity) {
return entity.getReportingTasks();
}
@Override
protected String getComponentId(ReportingTaskDTO dto) {
return dto.getId();
}
@Override
protected void mergeResponses(ReportingTaskDTO clientDto, Map<NodeIdentifier, ReportingTaskDTO> dtoMap, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses) {
new ReportingTaskEndpointMerger().mergeResponses(clientDto, dtoMap, successfulResponses, problematicResponses);
}
}

View File

@ -0,0 +1,222 @@
/*
* 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.ArrayList;
import java.util.Date;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.TimeUnit;
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.protocol.NodeIdentifier;
import org.apache.nifi.controller.status.history.ConnectionStatusDescriptor;
import org.apache.nifi.controller.status.history.MetricDescriptor;
import org.apache.nifi.controller.status.history.ProcessGroupStatusDescriptor;
import org.apache.nifi.controller.status.history.ProcessorStatusDescriptor;
import org.apache.nifi.controller.status.history.RemoteProcessGroupStatusDescriptor;
import org.apache.nifi.controller.status.history.StandardStatusSnapshot;
import org.apache.nifi.controller.status.history.StatusHistoryUtil;
import org.apache.nifi.controller.status.history.StatusSnapshot;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.api.dto.status.NodeStatusSnapshotsDTO;
import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
import org.apache.nifi.web.api.dto.status.StatusSnapshotDTO;
import org.apache.nifi.web.api.entity.StatusHistoryEntity;
public class StatusHistoryEndpointMerger implements EndpointResponseMerger {
public static final Pattern PROCESSOR_STATUS_HISTORY_URI_PATTERN = Pattern.compile("/nifi-api/flow/processors/[a-f0-9\\-]{36}/status/history");
public static final Pattern PROCESS_GROUP_STATUS_HISTORY_URI_PATTERN = Pattern.compile("/nifi-api/flow/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/status/history");
public static final Pattern REMOTE_PROCESS_GROUP_STATUS_HISTORY_URI_PATTERN = Pattern.compile("/nifi-api/flow/remote-process-groups/[a-f0-9\\-]{36}/status/history");
public static final Pattern CONNECTION_STATUS_HISTORY_URI_PATTERN = Pattern.compile("/nifi-api/flow/connections/[a-f0-9\\-]{36}/status/history");
private final long componentStatusSnapshotMillis;
public StatusHistoryEndpointMerger() {
final NiFiProperties properties = NiFiProperties.getInstance();
final String snapshotFrequency = properties.getProperty(NiFiProperties.COMPONENT_STATUS_SNAPSHOT_FREQUENCY, NiFiProperties.DEFAULT_COMPONENT_STATUS_SNAPSHOT_FREQUENCY);
long snapshotMillis;
try {
snapshotMillis = FormatUtils.getTimeDuration(snapshotFrequency, TimeUnit.MILLISECONDS);
} catch (final Exception e) {
snapshotMillis = FormatUtils.getTimeDuration(NiFiProperties.DEFAULT_COMPONENT_STATUS_SNAPSHOT_FREQUENCY, TimeUnit.MILLISECONDS);
}
componentStatusSnapshotMillis = snapshotMillis;
}
private Map<String, MetricDescriptor<?>> getMetricDescriptors(final URI uri) {
final String path = uri.getPath();
final Map<String, MetricDescriptor<?>> metricDescriptors = new HashMap<>();
if (PROCESSOR_STATUS_HISTORY_URI_PATTERN.matcher(path).matches()) {
for (final ProcessorStatusDescriptor descriptor : ProcessorStatusDescriptor.values()) {
metricDescriptors.put(descriptor.getField(), descriptor.getDescriptor());
}
} else if (PROCESS_GROUP_STATUS_HISTORY_URI_PATTERN.matcher(path).matches()) {
for (final ProcessGroupStatusDescriptor descriptor : ProcessGroupStatusDescriptor.values()) {
metricDescriptors.put(descriptor.getField(), descriptor.getDescriptor());
}
} else if (REMOTE_PROCESS_GROUP_STATUS_HISTORY_URI_PATTERN.matcher(path).matches()) {
for (final RemoteProcessGroupStatusDescriptor descriptor : RemoteProcessGroupStatusDescriptor.values()) {
metricDescriptors.put(descriptor.getField(), descriptor.getDescriptor());
}
} else if (CONNECTION_STATUS_HISTORY_URI_PATTERN.matcher(path).matches()) {
for (final ConnectionStatusDescriptor descriptor : ConnectionStatusDescriptor.values()) {
metricDescriptors.put(descriptor.getField(), descriptor.getDescriptor());
}
}
return metricDescriptors;
}
@Override
public boolean canHandle(URI uri, String method) {
if (!"GET".equalsIgnoreCase(method)) {
return false;
}
final Map<String, MetricDescriptor<?>> descriptors = getMetricDescriptors(uri);
return descriptors != null && !descriptors.isEmpty();
}
@Override
public NodeResponse merge(URI uri, String method, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses, NodeResponse clientResponse) {
final Map<String, MetricDescriptor<?>> metricDescriptors = getMetricDescriptors(uri);
final StatusHistoryEntity responseEntity = clientResponse.getClientResponse().getEntity(StatusHistoryEntity.class);
StatusHistoryDTO lastStatusHistory = null;
final List<NodeStatusSnapshotsDTO> nodeStatusSnapshots = new ArrayList<>(successfulResponses.size());
for (final NodeResponse nodeResponse : successfulResponses) {
final StatusHistoryEntity nodeResponseEntity = nodeResponse == clientResponse ? responseEntity : nodeResponse.getClientResponse().getEntity(StatusHistoryEntity.class);
final StatusHistoryDTO nodeStatus = nodeResponseEntity.getStatusHistory();
lastStatusHistory = nodeStatus;
final NodeIdentifier nodeId = nodeResponse.getNodeId();
final NodeStatusSnapshotsDTO nodeStatusSnapshot = new NodeStatusSnapshotsDTO();
nodeStatusSnapshot.setNodeId(nodeId.getId());
nodeStatusSnapshot.setAddress(nodeId.getApiAddress());
nodeStatusSnapshot.setApiPort(nodeId.getApiPort());
nodeStatusSnapshot.setStatusSnapshots(nodeStatus.getAggregateSnapshots());
nodeStatusSnapshots.add(nodeStatusSnapshot);
}
final StatusHistoryDTO clusterStatusHistory = new StatusHistoryDTO();
clusterStatusHistory.setAggregateSnapshots(mergeStatusHistories(nodeStatusSnapshots, metricDescriptors));
clusterStatusHistory.setGenerated(new Date());
clusterStatusHistory.setNodeSnapshots(nodeStatusSnapshots);
if (lastStatusHistory != null) {
clusterStatusHistory.setComponentDetails(lastStatusHistory.getComponentDetails());
clusterStatusHistory.setFieldDescriptors(lastStatusHistory.getFieldDescriptors());
}
final StatusHistoryEntity clusterEntity = new StatusHistoryEntity();
clusterEntity.setStatusHistory(clusterStatusHistory);
clusterEntity.setRevision(responseEntity.getRevision());
return new NodeResponse(clientResponse, clusterEntity);
}
private List<StatusSnapshotDTO> mergeStatusHistories(final List<NodeStatusSnapshotsDTO> nodeStatusSnapshots, final Map<String, MetricDescriptor<?>> metricDescriptors) {
// We want a Map<Date, List<StatusSnapshot>>, which is a Map of "normalized Date" (i.e., a time range, essentially)
// to all Snapshots for that time. The list will contain one snapshot for each node. However, we can have the case
// where the NCM has a different value for the componentStatusSnapshotMillis than the nodes have. In this case,
// we end up with multiple entries in the List<StatusSnapshot> for the same node/timestamp, which skews our aggregate
// results. In order to avoid this, we will use only the latest snapshot for a node that falls into the the time range
// of interest.
// To accomplish this, we have an intermediate data structure, which is a Map of "normalized Date" to an inner Map
// of Node Identifier to StatusSnapshot. We then will flatten this Map and aggregate the results.
final Map<Date, Map<String, StatusSnapshot>> dateToNodeSnapshots = new TreeMap<>();
// group status snapshot's for each node by date
for (final NodeStatusSnapshotsDTO nodeStatusSnapshot : nodeStatusSnapshots) {
for (final StatusSnapshotDTO snapshotDto : nodeStatusSnapshot.getStatusSnapshots()) {
final StatusSnapshot snapshot = createSnapshot(snapshotDto, metricDescriptors);
final Date normalizedDate = normalizeStatusSnapshotDate(snapshot.getTimestamp(), componentStatusSnapshotMillis);
Map<String, StatusSnapshot> nodeToSnapshotMap = dateToNodeSnapshots.get(normalizedDate);
if (nodeToSnapshotMap == null) {
nodeToSnapshotMap = new HashMap<>();
dateToNodeSnapshots.put(normalizedDate, nodeToSnapshotMap);
}
nodeToSnapshotMap.put(nodeStatusSnapshot.getNodeId(), snapshot);
}
}
// aggregate the snapshots by (normalized) timestamp
final Map<Date, List<StatusSnapshot>> snapshotsToAggregate = new TreeMap<>();
for (final Map.Entry<Date, Map<String, StatusSnapshot>> entry : dateToNodeSnapshots.entrySet()) {
final Date normalizedDate = entry.getKey();
final Map<String, StatusSnapshot> nodeToSnapshot = entry.getValue();
final List<StatusSnapshot> snapshotsForTimestamp = new ArrayList<>(nodeToSnapshot.values());
snapshotsToAggregate.put(normalizedDate, snapshotsForTimestamp);
}
final List<StatusSnapshotDTO> aggregatedSnapshots = aggregate(snapshotsToAggregate);
return aggregatedSnapshots;
}
private StatusSnapshot createSnapshot(final StatusSnapshotDTO snapshotDto, final Map<String, MetricDescriptor<?>> metricDescriptors) {
final StandardStatusSnapshot snapshot = new StandardStatusSnapshot();
snapshot.setTimestamp(snapshotDto.getTimestamp());
final Map<String, Long> metrics = snapshotDto.getStatusMetrics();
for (final Map.Entry<String, Long> entry : metrics.entrySet()) {
final String metricId = entry.getKey();
final Long value = entry.getValue();
final MetricDescriptor<?> descriptor = metricDescriptors.get(metricId);
if (descriptor != null) {
snapshot.addStatusMetric(descriptor, value);
}
}
return snapshot;
}
private List<StatusSnapshotDTO> aggregate(Map<Date, List<StatusSnapshot>> snapshotsToAggregate) {
// Aggregate the snapshots
final List<StatusSnapshotDTO> aggregatedSnapshotDtos = new ArrayList<>();
for (final Map.Entry<Date, List<StatusSnapshot>> entry : snapshotsToAggregate.entrySet()) {
final List<StatusSnapshot> snapshots = entry.getValue();
final StatusSnapshot reducedSnapshot = snapshots.get(0).getValueReducer().reduce(snapshots);
final StatusSnapshotDTO dto = new StatusSnapshotDTO();
dto.setTimestamp(reducedSnapshot.getTimestamp());
dto.setStatusMetrics(StatusHistoryUtil.createStatusSnapshotDto(reducedSnapshot).getStatusMetrics());
aggregatedSnapshotDtos.add(dto);
}
return aggregatedSnapshotDtos;
}
public static Date normalizeStatusSnapshotDate(final Date toNormalize, final long numMillis) {
final long time = toNormalize.getTime();
return new Date(time - time % numMillis);
}
}

View File

@ -0,0 +1,73 @@
/*
* 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.ArrayList;
import java.util.Map;
import java.util.regex.Pattern;
import org.apache.nifi.cluster.manager.StatusMerger;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.web.api.dto.NodeSystemDiagnosticsSnapshotDTO;
import org.apache.nifi.web.api.dto.SystemDiagnosticsDTO;
import org.apache.nifi.web.api.entity.SystemDiagnosticsEntity;
public class SystemDiagnosticsEndpointMerger extends AbstractNodeStatusEndpoint<SystemDiagnosticsEntity, SystemDiagnosticsDTO> {
public static final Pattern SYSTEM_DIAGNOSTICS_URI_PATTERN = Pattern.compile("/nifi-api/system-diagnostics");
@Override
public boolean canHandle(URI uri, String method) {
return "GET".equalsIgnoreCase(method) && SYSTEM_DIAGNOSTICS_URI_PATTERN.matcher(uri.getPath()).matches();
}
@Override
protected Class<SystemDiagnosticsEntity> getEntityClass() {
return SystemDiagnosticsEntity.class;
}
@Override
protected SystemDiagnosticsDTO getDto(SystemDiagnosticsEntity entity) {
return entity.getSystemDiagnostics();
}
@Override
protected void mergeResponses(SystemDiagnosticsDTO clientDto, Map<NodeIdentifier, SystemDiagnosticsDTO> dtoMap, NodeIdentifier selectedNodeId) {
final SystemDiagnosticsDTO mergedSystemDiagnostics = clientDto;
mergedSystemDiagnostics.setNodeSnapshots(new ArrayList<NodeSystemDiagnosticsSnapshotDTO>());
final NodeSystemDiagnosticsSnapshotDTO selectedNodeSnapshot = new NodeSystemDiagnosticsSnapshotDTO();
selectedNodeSnapshot.setSnapshot(clientDto.getAggregateSnapshot().clone());
selectedNodeSnapshot.setAddress(selectedNodeId.getApiAddress());
selectedNodeSnapshot.setApiPort(selectedNodeId.getApiPort());
selectedNodeSnapshot.setNodeId(selectedNodeId.getId());
mergedSystemDiagnostics.getNodeSnapshots().add(selectedNodeSnapshot);
for (final Map.Entry<NodeIdentifier, SystemDiagnosticsDTO> entry : dtoMap.entrySet()) {
final NodeIdentifier nodeId = entry.getKey();
final SystemDiagnosticsDTO toMerge = entry.getValue();
if (toMerge == clientDto) {
continue;
}
StatusMerger.merge(mergedSystemDiagnostics, toMerge, nodeId.getId(), nodeId.getApiAddress(), nodeId.getApiPort());
}
}
}

View File

@ -0,0 +1,122 @@
/*
* 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.replication;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
/**
* A response that is provided when an HTTP Request is made that must be federated to nodes in the cluster
*/
public interface AsyncClusterResponse {
/**
* @return the unique identifier of the request
*/
String getRequestIdentifier();
/**
* @return the HTTP Method that was used for the request
*/
String getMethod();
/**
* @return the Path of the URI that was used for the request
*/
String getURIPath();
/**
* @return a Set that contains the Node Identifier of each node to which the request was replicated
*/
Set<NodeIdentifier> getNodesInvolved();
/**
* @return a Set that contains the Node Identifier of each node for which the request has completed (either with
* a successful response or a timeout)
*/
Set<NodeIdentifier> getCompletedNodeIdentifiers();
/**
* @return a Set that contains the Node Response of each node for which the request has completed (either with a
* successful response or a timeout)
*/
Set<NodeResponse> getCompletedNodeResponses();
/**
* @return <code>true</code> if all nodes have responded (or timed out), <code>false</code> if still waiting on a response
* from one or more nodes
*/
boolean isComplete();
/**
* Indicates whether or not the request was created more than the specified amount of time ago.
* For example, if called via
* <code>isOlderThan(3, TimeUnit.SECONDS)</code>
* the method will return <code>true</code> if the request was created more than 3 seconds ago.
*
* @param time the amount of time to check
* @param timeUnit the associated time unit
* @return <code>true</code> if the request was created before (now - time)
*/
boolean isOlderThan(long time, TimeUnit timeUnit);
/**
* @return the {@link NodeResponse} that represents the merged result from all nodes, or <code>null</code> if this request
* is not yet complete
*
* @throws RuntimeException if the request could not be completed for some reason, a
* RuntimeException will be thrown that indicates why the request failed
*/
NodeResponse getMergedResponse();
/**
* Blocks until the request has completed and then returns the merged response
*
* @return the NodeResponse that represents the merged result from all nodes
*
* @throws InterruptedException if the thread is interrupted while waiting
* @throws RuntimeException if the request could not be completed for some reason, a
* RuntimeException will be thrown that indicates why the request failed
*/
NodeResponse awaitMergedResponse() throws InterruptedException;
/**
* Blocks until the request has completed or until the given timeout elapses. At that point, if the request has completed, then
* the merged response is returned. If the timeout elapses before the request completes, then <code>null</code> will be returned.
*
* @return the NodeResponse that represents the merged result from all nodes, or <code>null</code> if the given timeout elapses
* before the request completes
*
* @throws InterruptedException if the thread is interrupted while waiting
* @throws RuntimeException if the request could not be completed for some reason, a
* RuntimeException will be thrown that indicates why the request failed
*
*/
NodeResponse awaitMergedResponse(long timeout, TimeUnit timeUnit) throws InterruptedException;
/**
* Returns the NodeResponse that represents the individual response from the node with the given identifier
*
* @param nodeId the ID of the node whose response is to be returned
* @return the NodeResponse from the node with the given identifier, or <code>null</code> if there is no response yet from the given node
*/
NodeResponse getNodeResponse(NodeIdentifier nodeId);
}

View File

@ -0,0 +1,22 @@
/*
* 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.replication;
public interface CompletionCallback {
void onCompletion(AsyncClusterResponse response);
}

View File

@ -0,0 +1,39 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.coordination.http.replication;
import java.util.Set;
import org.apache.nifi.cluster.manager.NodeResponse;
/**
* A callback that can be registered to be called after an HTTP Request is replicated and all nodes'
* responses have been accounted for (either successfully or not)
*/
public interface RequestCompletionCallback {
/**
* Called after all NodeResponse objects have been gathered for the request
*
* @param uriPath the path of the request URI
* @param method the HTTP method of the request
* @param nodeResponses the NodeResponse for each node that the request was replicated to
*/
void afterRequest(String uriPath, String method, Set<NodeResponse> nodeResponses);
}

View File

@ -0,0 +1,74 @@
/*
* 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.replication;
import java.net.URI;
import java.util.Map;
import java.util.Set;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
public interface RequestReplicator {
public static final String REQUEST_TRANSACTION_ID = "X-RequestTransactionId";
/**
* Starts the instance for replicating requests. Calling this method on an already started instance has no effect.
*/
void start();
/**
* Stops the instance from replicating requests. Calling this method on a stopped instance has no effect.
*/
void stop();
/**
* @return true if the instance is started; false otherwise.
*/
boolean isRunning();
/**
* Requests are sent to each node in the given set of Node Identifiers. The returned AsyncClusterResponse object will contain
* the results that are immediately available, as well as an identifier for obtaining an updated result later.
*
* 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 an AsyncClusterResponse that indicates the current status of the request and provides an identifier for obtaining an updated response later
*/
AsyncClusterResponse replicate(Set<NodeIdentifier> nodeIds, String method, URI uri, Object entity, Map<String, String> headers);
/**
* <p>
* Returns an AsyncClusterResponse that provides the most up-to-date status of the request with the given identifier.
* If the request is finished, meaning that all nodes in the cluster have reported back their status or have timed out,
* then the response will be removed and any subsequent calls to obtain the response with the same identifier will return
* <code>null</code>. If the response is not complete, the method may be called again at some point in the future in order
* to check again if the request has completed.
* </p>
*
* @param requestIdentifier the identifier of the request to obtain a response for
* @return an AsyncClusterResponse that provides the most up-to-date status of the request with the given identifier, or <code>null</code> if
* no request exists with the given identifier
*/
AsyncClusterResponse getClusterResponse(String requestIdentifier);
}

View File

@ -0,0 +1,88 @@
/*
* 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.replication;
import java.util.HashSet;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
public class ResponseUtils {
/**
* Finds the Node Identifier for all nodes that had a 'slow' response time. A 'slow' response time
* is defined as being more than X standard deviations from the mean response time, where X is the
* given Standard Deviation Multiple
*
* @param response the AsyncClusterResponse to base the calculations off of
* @param stdDeviationMultiple the number of standard deviations that a response time must be away from the mean in order
* to be considered 'slow'
*
* @return a Set of all Node Identifiers that took a long time to respond
*/
public static Set<NodeIdentifier> findLongResponseTimes(final AsyncClusterResponse response, final double stdDeviationMultiple) {
final Set<NodeIdentifier> slowResponses = new HashSet<>();
if (response.isOlderThan(2, TimeUnit.SECONDS)) {
// If the response is older than 2 seconds, determines if any node took a long time to respond.
final Set<NodeIdentifier> completedIds = response.getCompletedNodeIdentifiers();
if (completedIds.size() < 2) {
return slowResponses;
}
long requestMillisSum = 0L;
int numNodes = 0;
for (final NodeIdentifier nodeId : completedIds) {
final long requestMillis = response.getNodeResponse(nodeId).getRequestDuration(TimeUnit.NANOSECONDS);
if (requestMillis < 0) {
continue;
}
requestMillisSum += requestMillis;
numNodes++;
}
if (numNodes < 2) {
return slowResponses;
}
final double mean = requestMillisSum / numNodes;
double differenceSquaredSum = 0D;
for (final NodeIdentifier nodeId : completedIds) {
final long requestMillis = response.getNodeResponse(nodeId).getRequestDuration(TimeUnit.NANOSECONDS);
final double differenceSquared = Math.pow(mean - requestMillis, 2);
differenceSquaredSum += differenceSquared;
}
final double meanOfDifferenceSquared = differenceSquaredSum / numNodes;
final double stdDev = Math.pow(meanOfDifferenceSquared, 0.5D);
final double longTimeThreshold = mean + stdDev * stdDeviationMultiple;
for (final NodeIdentifier nodeId : completedIds) {
final long requestMillis = response.getNodeResponse(nodeId).getRequestDuration(TimeUnit.NANOSECONDS);
if (requestMillis > longTimeThreshold) {
slowResponses.add(nodeId);
}
}
}
return slowResponses;
}
}

View File

@ -0,0 +1,263 @@
/*
* 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.replication;
import java.net.URI;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors;
import org.apache.nifi.cluster.coordination.http.HttpResponseMerger;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class StandardAsyncClusterResponse implements AsyncClusterResponse {
private static final Logger logger = LoggerFactory.getLogger(StandardAsyncClusterResponse.class);
private final String id;
private final Set<NodeIdentifier> nodeIds;
private final URI uri;
private final String method;
private final HttpResponseMerger responseMerger;
private final CompletionCallback completionCallback;
private final Runnable completedResultFetchedCallback;
private final long creationTimeNanos;
private final Map<NodeIdentifier, ResponseHolder> responseMap = new HashMap<>();
private final AtomicInteger requestsCompleted = new AtomicInteger(0);
private NodeResponse mergedResponse; // guarded by synchronizing on this
private RuntimeException failure; // guarded by synchronizing on this
public StandardAsyncClusterResponse(final String id, final URI uri, final String method, final Set<NodeIdentifier> nodeIds,
final HttpResponseMerger responseMerger, final CompletionCallback completionCallback, final Runnable completedResultFetchedCallback) {
this.id = id;
this.nodeIds = Collections.unmodifiableSet(new HashSet<>(nodeIds));
this.uri = uri;
this.method = method;
creationTimeNanos = System.nanoTime();
for (final NodeIdentifier nodeId : nodeIds) {
responseMap.put(nodeId, new ResponseHolder(creationTimeNanos));
}
this.responseMerger = responseMerger;
this.completionCallback = completionCallback;
this.completedResultFetchedCallback = completedResultFetchedCallback;
}
@Override
public String getRequestIdentifier() {
return id;
}
@Override
public Set<NodeIdentifier> getNodesInvolved() {
return nodeIds;
}
@Override
public Set<NodeIdentifier> getCompletedNodeIdentifiers() {
return responseMap.entrySet().stream()
.filter(entry -> entry.getValue().isComplete())
.map(entry -> entry.getKey())
.collect(Collectors.toSet());
}
@Override
public Set<NodeResponse> getCompletedNodeResponses() {
return responseMap.values().stream()
.filter(responseHolder -> responseHolder.isComplete())
.map(responseHolder -> responseHolder.getResponse())
.collect(Collectors.toSet());
}
@Override
public boolean isOlderThan(final long time, final TimeUnit timeUnit) {
final long nanos = timeUnit.toNanos(time);
final long threshold = System.nanoTime() - nanos;
return creationTimeNanos < threshold;
}
@Override
public boolean isComplete() {
return getMergedResponse() != null;
}
@Override
public String getMethod() {
return method;
}
@Override
public String getURIPath() {
return uri.getPath();
}
@Override
public NodeResponse getMergedResponse() {
return getMergedResponse(true);
}
public synchronized NodeResponse getMergedResponse(final boolean triggerCallback) {
if (failure != null) {
throw failure;
}
if (mergedResponse != null) {
if (triggerCallback && completedResultFetchedCallback != null) {
completedResultFetchedCallback.run();
}
return mergedResponse;
}
if (requestsCompleted.get() < responseMap.size()) {
return null;
}
final Set<NodeResponse> nodeResponses = responseMap.values().stream().map(p -> p.getResponse()).collect(Collectors.toSet());
mergedResponse = responseMerger.mergeResponses(uri, method, nodeResponses);
logger.debug("Notifying all that merged response is complete for {}", id);
this.notifyAll();
if (triggerCallback && completedResultFetchedCallback != null) {
completedResultFetchedCallback.run();
}
return mergedResponse;
}
@Override
public NodeResponse awaitMergedResponse() throws InterruptedException {
synchronized (this) {
while (getMergedResponse(false) == null) {
logger.debug("Waiting indefinitely for merged response to be complete for {}", id);
this.wait();
}
}
return getMergedResponse(true);
}
@Override
public NodeResponse awaitMergedResponse(final long timeout, final TimeUnit timeUnit) throws InterruptedException {
if (timeout < 0) {
throw new IllegalArgumentException();
}
final long maxTime = System.nanoTime() + timeUnit.toNanos(timeout);
synchronized (this) {
while (getMergedResponse(false) == null) {
final long nanosToWait = maxTime - System.nanoTime();
if (nanosToWait < 0) {
return getMergedResponse(true);
}
final long millis = TimeUnit.NANOSECONDS.toMillis(nanosToWait);
final int nanos = (int) (nanosToWait - TimeUnit.MILLISECONDS.toNanos(millis));
logger.debug("Waiting {} millis and {} nanos for merged response to be complete for {}", millis, nanos, id);
this.wait(millis, nanos);
}
}
return getMergedResponse(true);
}
@Override
public NodeResponse getNodeResponse(final NodeIdentifier nodeId) {
final ResponseHolder request = responseMap.get(nodeId);
return request == null ? null : request.getResponse();
}
void add(final NodeResponse nodeResponse) {
final ResponseHolder responseHolder = responseMap.get(nodeResponse.getNodeId());
if (responseHolder == null) {
throw new IllegalStateException("Node " + nodeResponse.getNodeId() + " is not known for this request");
}
responseHolder.setResponse(nodeResponse);
final int completedCount = requestsCompleted.incrementAndGet();
logger.debug("Received response {} out of {} for {} from {}", completedCount, responseMap.size(), id, nodeResponse.getNodeId());
if (completedCount == responseMap.size()) {
logger.debug("Notifying all that merged response is ready for {}", id);
synchronized (this) {
this.notifyAll();
}
if (completionCallback != null) {
completionCallback.onCompletion(this);
}
}
}
synchronized void setFailure(final RuntimeException failure) {
this.failure = failure;
notifyAll();
if (completionCallback != null) {
completionCallback.onCompletion(this);
}
}
@Override
public String toString() {
return "StandardAsyncClusterResponse[id=" + id + ", uri=" + uri + ", method=" + method + ", failure=" + (failure != null)
+ ", responses=" + getCompletedNodeIdentifiers().size() + "/" + responseMap.size() + "]";
}
private static class ResponseHolder {
private final long nanoStart;
private long requestNanos;
private NodeResponse response;
public ResponseHolder(final long startNanos) {
this.nanoStart = startNanos;
}
public synchronized void setResponse(final NodeResponse response) {
this.response = response;
this.requestNanos = System.nanoTime() - nanoStart;
}
public synchronized NodeResponse getResponse() {
return response;
}
public synchronized boolean isComplete() {
return response != null;
}
@SuppressWarnings("unused")
public long getRequestDuration(final TimeUnit timeUnit) {
return timeUnit.toNanos(requestNanos);
}
}
}

View File

@ -0,0 +1,672 @@
/*
* 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.replication;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.LongSummaryStatistics;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Function;
import java.util.stream.Collectors;
import javax.ws.rs.HttpMethod;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.MultivaluedMap;
import org.apache.nifi.cluster.context.ClusterContext;
import org.apache.nifi.cluster.context.ClusterContextImpl;
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.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.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;
import org.apache.nifi.reporting.Severity;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.web.OptimisticLockingManager;
import org.apache.nifi.web.util.WebUtils;
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.WebResource;
import com.sun.jersey.api.client.config.ClientConfig;
import com.sun.jersey.api.client.filter.GZIPContentEncodingFilter;
import com.sun.jersey.core.util.MultivaluedMapImpl;
public class ThreadPoolRequestReplicator implements RequestReplicator {
/**
* The HTTP header to store a cluster context. An example of what may be stored in the context is a node's
* auditable actions in response to a cluster request. The cluster context is serialized
* using Java's serialization mechanism and hex encoded.
*/
static final String CLUSTER_CONTEXT_HTTP_HEADER = "X-ClusterContext";
/**
* The HTTP header that the NCM specifies to ask a node if they are able to process a given request. The value
* is always 150-NodeContinue. The node will respond with 150 CONTINUE if it is able to
* process the request, 417 EXPECTATION_FAILED otherwise.
*/
static final String NCM_EXPECTS_HTTP_HEADER = "X-NcmExpects";
static final String NODE_CONTINUE = "150-NodeContinue";
static final int NODE_CONTINUE_STATUS_CODE = 150;
private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(ThreadPoolRequestReplicator.class));
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 OptimisticLockingManager lockingManager;
private final DataFlowManagementService dfmService;
private ExecutorService executorService;
private ScheduledExecutorService maintenanceExecutor;
private final ConcurrentMap<String, StandardAsyncClusterResponse> responseMap = new ConcurrentHashMap<>();
private final ConcurrentMap<NodeIdentifier, AtomicInteger> sequentialLongRequestCounts = new ConcurrentHashMap<>();
/**
* Creates an instance using a connection timeout and read timeout of 3 seconds
*
* @param numThreads the number of threads to use when parallelizing requests
* @param client a client for making requests
* @param clusterCoordinator the cluster coordinator to use for interacting with node statuses
* @param callback a callback that will be called whenever all of the responses have been gathered for a request. May be null.
* @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 OptimisticLockingManager lockingManager, final DataFlowManagementService dfmService) {
this(numThreads, client, clusterCoordinator, "3 sec", "3 sec", callback, eventReporter, null, lockingManager, dfmService);
}
/**
* Creates an instance.
*
* @param numThreads the number of threads to use when parallelizing requests
* @param client a client for making requests
* @param clusterCoordinator the cluster coordinator to use for interacting with node statuses
* @param connectionTimeout the connection timeout specified in milliseconds
* @param readTimeout the read timeout specified in milliseconds
* @param callback a callback that will be called whenever all of the responses have been gathered for a request. May be null.
* @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 OptimisticLockingManager lockingManager, final DataFlowManagementService dfmService) {
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.eventReporter = eventReporter;
this.callback = callback;
this.lockingManager = lockingManager;
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;
}
executorService = Executors.newFixedThreadPool(numThreads);
maintenanceExecutor = Executors.newScheduledThreadPool(1, new ThreadFactory() {
@Override
public Thread newThread(final Runnable r) {
final Thread t = Executors.defaultThreadFactory().newThread(r);
t.setDaemon(true);
t.setName(ThreadPoolRequestReplicator.class.getSimpleName() + " Maintenance Thread");
return t;
}
});
maintenanceExecutor.scheduleWithFixedDelay(new PurgeExpiredRequestsTask(), 3, 3, TimeUnit.SECONDS);
}
@Override
public boolean isRunning() {
return executorService != null && !executorService.isShutdown();
}
@Override
public void stop() {
if (!isRunning()) {
return;
}
executorService.shutdown();
maintenanceExecutor.shutdown();
}
@Override
public AsyncClusterResponse replicate(Set<NodeIdentifier> nodeIds, String method, URI uri, Object entity, Map<String, String> headers) {
return replicate(nodeIds, method, uri, entity, headers, true, null);
}
/**
* Replicates the request to all nodes in the given set of node identifiers
*
* @param nodeIds the NodeIdentifiers that identify which nodes to send the request to
* @param method the HTTP method to use
* @param uri the URI to send the request to
* @param entity the entity to use
* @param headers the HTTP Headers
* @param performVerification whether or not to use 2-phase commit to verify that all nodes can handle the request. Ignored if request is not mutable.
* @param response the response to update with the results
*
* @return an AsyncClusterResponse that can be used to obtain the result
*/
private AsyncClusterResponse replicate(Set<NodeIdentifier> nodeIds, String method, URI uri, Object entity, Map<String, String> headers, boolean performVerification,
StandardAsyncClusterResponse response) {
// state validation
Objects.requireNonNull(nodeIds);
Objects.requireNonNull(method);
Objects.requireNonNull(uri);
Objects.requireNonNull(entity);
Objects.requireNonNull(headers);
if (nodeIds.isEmpty()) {
throw new IllegalArgumentException("Cannot replicate request to 0 nodes");
}
if (performVerification) {
verifyState(method, uri.getPath());
}
final int numRequests = responseMap.size();
if (numRequests >= MAX_CONCURRENT_REQUESTS) {
throw new IllegalStateException("There are too many outstanding HTTP requests with a total " + numRequests + " outstanding requests");
}
// create the request objects and replicate to all nodes
final String requestId = UUID.randomUUID().toString();
final CompletionCallback completionCallback = clusterResponse -> onCompletedResponse(requestId);
final Runnable responseConsumedCallback = () -> onResponseConsumed(requestId);
// create a response object if one was not already passed to us
if (response == null) {
response = new StandardAsyncClusterResponse(requestId, uri, method, nodeIds,
responseMerger, completionCallback, responseConsumedCallback);
responseMap.put(requestId, response);
}
// Update headers to indicate the current revision so that we can
// prevent multiple users changing the flow at the same time
final Map<String, String> updatedHeaders = new HashMap<>(headers);
updatedHeaders.put(REQUEST_TRANSACTION_ID, UUID.randomUUID().toString());
// 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
// issue the request. This is all handled by calling performVerification, which will replicate
// the 'vote' request to all nodes and then if successful will call back into this method to
// replicate the actual request.
final boolean mutableRequest = isMutableRequest(method, uri.getPath());
if (mutableRequest && performVerification) {
performVerification(nodeIds, method, uri, entity, updatedHeaders, response);
return response;
}
// Callback function for generating a NodeHttpRequestCallable that can be used to perform the work
final StandardAsyncClusterResponse finalResponse = response;
NodeRequestCompletionCallback nodeCompletionCallback = nodeResponse -> {
logger.debug("Received response from {} for {} {}", nodeResponse.getNodeId(), method, uri.getPath());
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);
replicateRequest(nodeIds, uri.getScheme(), uri.getPath(), requestFactory, updatedHeaders);
// TODO: Must handle revisions!!
return response;
}
private void setRevision(final Map<String, String> headers) {
final ClusterContext clusterCtx = new ClusterContextImpl();
clusterCtx.setRequestSentByClusterManager(true); // indicate request is sent from cluster manager
clusterCtx.setRevision(lockingManager.getLastModification().getRevision());
// serialize cluster context and add to request header
final String serializedClusterCtx = WebUtils.serializeObjectToHex(clusterCtx);
headers.put(CLUSTER_CONTEXT_HTTP_HEADER, serializedClusterCtx);
}
private void performVerification(Set<NodeIdentifier> nodeIds, String method, URI uri, Object entity, Map<String, String> headers, StandardAsyncClusterResponse clusterResponse) {
logger.debug("Verifying that mutable request {} {} can be made", method, uri.getPath());
final Map<String, String> updatedHeaders = new HashMap<>(headers);
updatedHeaders.put(NCM_EXPECTS_HTTP_HEADER, NODE_CONTINUE);
final int numNodes = nodeIds.size();
final NodeRequestCompletionCallback completionCallback = new NodeRequestCompletionCallback() {
final Set<NodeResponse> nodeResponses = Collections.synchronizedSet(new HashSet<>());
@Override
public void onCompletion(final NodeResponse nodeResponse) {
// Add the node response to our collection.
nodeResponses.add(nodeResponse);
try {
// If we have all of the node responses, then we can verify the responses
// and if good replicate the original request to all of the nodes.
if (nodeResponses.size() == numNodes) {
// Check if we have any requests that do not have a 150-Continue status code.
final long dissentingCount = nodeResponses.stream().filter(p -> p.getStatus() != NODE_CONTINUE_STATUS_CODE).count();
// If all nodes responded with 150-Continue, then we can replicate the original request
// to all nodes and we are finished.
if (dissentingCount == 0) {
logger.debug("Received verification from all {} nodes that mutable request {} {} can be made", numNodes, method, uri.getPath());
replicate(nodeIds, method, uri, entity, headers, false, clusterResponse);
return;
}
// Add a NodeResponse for each node to the Cluster Response
// Check that all nodes responded successfully.
for (final NodeResponse response : nodeResponses) {
if (response.getStatus() != NODE_CONTINUE_STATUS_CODE) {
final ClientResponse clientResponse = response.getClientResponse();
final RuntimeException failure;
if (clientResponse == null) {
failure = new IllegalClusterStateException("Node " + response.getNodeId()
+ " is unable to fulfill this request due to: Unexpected Response Code " + response.getStatus());
logger.info("Received a status of {} from {} for request {} {} when performing first stage of two-stage commit. "
+ "Will respond with CONFLICT response and action will not occur",
response.getStatus(), response.getNodeId(), method, uri.getPath());
} else {
final String nodeExplanation = clientResponse.getEntity(String.class);
failure = new IllegalClusterStateException("Node " + response.getNodeId() + " is unable to fulfill this request due to: "
+ nodeExplanation, response.getThrowable());
logger.info("Received a status of {} from {} for request {} {} when performing first stage of two-stage commit. "
+ "Will respond with CONFLICT response and action will not occur. Node explanation: {}",
response.getStatus(), response.getNodeId(), method, uri.getPath(), nodeExplanation);
}
clusterResponse.setFailure(failure);
break;
}
}
}
} catch (final Exception e) {
clusterResponse.add(new NodeResponse(nodeResponse.getNodeId(), method, uri, e));
// If there was a problem, we need to ensure that we add all of the other nodes' responses
// to the Cluster Response so that the Cluster Response is complete.
for (final NodeResponse otherResponse : nodeResponses) {
if (otherResponse.getNodeId().equals(nodeResponse.getNodeId())) {
continue;
}
clusterResponse.add(otherResponse);
}
}
}
};
// 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);
// replicate the 'verification request' to all nodes
replicateRequest(nodeIds, uri.getScheme(), uri.getPath(), requestFactory, updatedHeaders);
}
@Override
public AsyncClusterResponse getClusterResponse(final String identifier) {
final AsyncClusterResponse response = responseMap.get(identifier);
if (response == null) {
return null;
}
return response;
}
// Visible for testing - overriding this method makes it easy to verify behavior without actually making any web requests
protected NodeResponse replicateRequest(final WebResource.Builder resourceBuilder, final NodeIdentifier nodeId, final String method, final URI uri, final String requestId) {
final ClientResponse clientResponse;
final long startNanos = System.nanoTime();
switch (method.toUpperCase()) {
case HttpMethod.DELETE:
clientResponse = resourceBuilder.delete(ClientResponse.class);
break;
case HttpMethod.GET:
clientResponse = resourceBuilder.get(ClientResponse.class);
break;
case HttpMethod.HEAD:
clientResponse = resourceBuilder.head();
break;
case HttpMethod.OPTIONS:
clientResponse = resourceBuilder.options(ClientResponse.class);
break;
case HttpMethod.POST:
clientResponse = resourceBuilder.post(ClientResponse.class);
break;
case HttpMethod.PUT:
clientResponse = resourceBuilder.put(ClientResponse.class);
break;
default:
throw new IllegalArgumentException("HTTP Method '" + method + "' not supported for request replication.");
}
return new NodeResponse(nodeId, method, uri, clientResponse, System.nanoTime() - startNanos, requestId);
}
private boolean isMutableRequest(final String method, final String uriPath) {
switch (method.toUpperCase()) {
case HttpMethod.GET:
case HttpMethod.HEAD:
case HttpMethod.OPTIONS:
return false;
default:
return true;
}
}
/**
* Verifies that the cluster is in a state that will allow requests to be made using the given HTTP Method and URI path
*
* @param httpMethod the HTTP Method
* @param uriPath the URI Path
*
* @throw IllegalClusterStateException if the cluster is not in a state that allows a request to made to the given URI Path using the given HTTP Method
*/
private void verifyState(final String httpMethod, final String uriPath) {
final boolean mutableRequest = HttpMethod.DELETE.equals(httpMethod) || HttpMethod.POST.equals(httpMethod) || HttpMethod.PUT.equals(httpMethod);
// check that the request can be applied
if (mutableRequest) {
final Map<NodeConnectionState, List<NodeIdentifier>> connectionStates = clusterCoordinator.getConnectionStates();
if (connectionStates.containsKey(NodeConnectionState.DISCONNECTED) || connectionStates.containsKey(NodeConnectionState.DISCONNECTING)) {
throw new DisconnectedNodeMutableRequestException("Received a mutable request [" + httpMethod + " " + uriPath + "] while a node is disconnected from the cluster");
}
if (connectionStates.containsKey(NodeConnectionState.CONNECTING)) {
// if any node is connecting and a request can change the flow, then we throw an exception
throw new ConnectingNodeMutableRequestException("Received a mutable request [" + httpMethod + " " + uriPath + "] while a node is trying to connect to the cluster");
}
}
}
private void onResponseConsumed(final String requestId) {
final AsyncClusterResponse response = responseMap.remove(requestId);
if (response != null && logger.isDebugEnabled()) {
logTimingInfo(response);
}
}
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());
} catch (final Exception e) {
logger.warn("Completed request {} {} but failed to properly handle the Request Completion Callback due to {}",
response.getMethod(), response.getURIPath(), e.toString());
logger.warn("", e);
}
}
// 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);
for (final NodeIdentifier nodeId : response.getNodesInvolved()) {
final AtomicInteger counter = sequentialLongRequestCounts.computeIfAbsent(nodeId, id -> new AtomicInteger(0));
if (slowResponseNodes.contains(nodeId)) {
final int sequentialLongRequests = counter.incrementAndGet();
if (sequentialLongRequests >= 3) {
final String message = "Response time from " + nodeId + " was slow for each of the last 3 requests made. "
+ "To see more information about timing, enable DEBUG logging for " + logger.getName();
logger.warn(message);
if (eventReporter != null) {
eventReporter.reportEvent(Severity.WARNING, "Node Response Time", message);
}
counter.set(0);
}
} else {
counter.set(0);
}
}
}
private void logTimingInfo(final AsyncClusterResponse response) {
// Calculate min, max, mean for the requests
final LongSummaryStatistics stats = response.getNodesInvolved().stream()
.map(p -> response.getNodeResponse(p).getRequestDuration(TimeUnit.MILLISECONDS))
.collect(Collectors.summarizingLong(Long::longValue));
final StringBuilder sb = new StringBuilder();
sb.append("Node Responses for ").append(response.getMethod()).append(" ").append(response.getURIPath()).append(" (Request ID ").append(response.getRequestIdentifier()).append("):\n");
for (final NodeIdentifier node : response.getNodesInvolved()) {
sb.append(node).append(": ").append(response.getNodeResponse(node).getRequestDuration(TimeUnit.MILLISECONDS)).append(" millis\n");
}
logger.debug("For {} {} (Request ID {}), minimum response time = {}, max = {}, average = {} ms",
response.getMethod(), response.getURIPath(), response.getRequestIdentifier(), stats.getMin(), stats.getMax(), stats.getAverage());
logger.debug(sb.toString());
}
private void replicateRequest(final Set<NodeIdentifier> nodeIds, final String scheme,
final String path, final Function<NodeIdentifier, NodeHttpRequest> callableFactory, final Map<String, String> headers) {
if (nodeIds.isEmpty()) {
return; // return quickly for trivial case
}
// submit the requests to the nodes
final String requestId = UUID.randomUUID().toString();
headers.put(WebClusterManager.REQUEST_ID_HEADER, requestId);
for (final NodeIdentifier nodeId : nodeIds) {
final NodeHttpRequest callable = callableFactory.apply(nodeId);
executorService.submit(callable);
}
}
private URI createURI(final URI exampleUri, final NodeIdentifier nodeId) {
return createURI(exampleUri.getScheme(), nodeId.getApiAddress(), nodeId.getApiPort(), exampleUri.getPath());
}
private URI createURI(final String scheme, final String nodeApiAddress, final int nodeApiPort, final String path) {
try {
return new URI(scheme, null, nodeApiAddress, nodeApiPort, path, null, null);
} catch (final URISyntaxException e) {
throw new UriConstructionException(e);
}
}
/**
* A Callable for making an HTTP request to a single node and returning its response.
*/
private class NodeHttpRequest implements Runnable {
private final NodeIdentifier nodeId;
private final String method;
private final URI uri;
private final Object entity;
private final Map<String, String> headers = new HashMap<>();
private final NodeRequestCompletionCallback callback;
private NodeHttpRequest(final NodeIdentifier nodeId, final String method,
final URI uri, final Object entity, final Map<String, String> headers, final NodeRequestCompletionCallback callback) {
this.nodeId = nodeId;
this.method = method;
this.uri = uri;
this.entity = entity;
this.headers.putAll(headers);
this.callback = callback;
}
@Override
public void run() {
NodeResponse nodeResponse;
try {
// create and send the request
final WebResource.Builder resourceBuilder = createResourceBuilder();
final String requestId = headers.get("x-nifi-request-id");
logger.debug("Replicating request {} {} to {}", method, uri.getPath(), nodeId);
nodeResponse = replicateRequest(resourceBuilder, nodeId, method, uri, requestId);
} catch (final Exception e) {
nodeResponse = new NodeResponse(nodeId, method, uri, e);
logger.warn("Failed to replicate request {} {} to {} due to {}", method, uri.getPath(), nodeId, e);
logger.warn("", e);
}
if (callback != null) {
logger.debug("Request {} {} completed for {}", method, uri.getPath(), nodeId);
callback.onCompletion(nodeResponse);
}
}
@SuppressWarnings({"rawtypes", "unchecked"})
private WebResource.Builder createResourceBuilder() {
// convert parameters to a more convenient data structure
final MultivaluedMap<String, String> map = new MultivaluedMapImpl();
if (entity instanceof MultivaluedMap) {
map.putAll((Map) entity);
}
// create the resource
WebResource resource = client.resource(uri);
if (responseMerger.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;
}
}
private static interface NodeRequestCompletionCallback {
void onCompletion(NodeResponse nodeResponse);
}
private class PurgeExpiredRequestsTask implements Runnable {
@Override
public void run() {
final Set<String> expiredRequestIds = ThreadPoolRequestReplicator.this.responseMap.entrySet().stream()
.filter(entry -> entry.getValue().isOlderThan(30, TimeUnit.SECONDS)) // older than 30 seconds
.filter(entry -> entry.getValue().isComplete()) // is complete
.map(entry -> entry.getKey()) // get the request id
.collect(Collectors.toSet());
expiredRequestIds.forEach(id -> onResponseConsumed(id));
}
}
}

View File

@ -62,15 +62,9 @@ public class StatusMerger {
return;
}
target.setActiveRemotePortCount(target.getActiveRemotePortCount() + toMerge.getActiveRemotePortCount());
target.setActiveThreadCount(target.getActiveThreadCount() + toMerge.getActiveThreadCount());
target.setBytesQueued(target.getBytesQueued() + toMerge.getBytesQueued());
target.setDisabledCount(target.getDisabledCount() + toMerge.getDisabledCount());
target.setFlowFilesQueued(target.getFlowFilesQueued() + toMerge.getFlowFilesQueued());
target.setInactiveRemotePortCount(target.getInactiveRemotePortCount() + toMerge.getInactiveRemotePortCount());
target.setInvalidCount(target.getInvalidCount() + toMerge.getInvalidCount());
target.setRunningCount(target.getRunningCount() + toMerge.getRunningCount());
target.setStoppedCount(target.getStoppedCount() + toMerge.getStoppedCount());
target.setBulletins(mergeBulletins(target.getBulletins(), toMerge.getBulletins()));
target.setControllerServiceBulletins(mergeBulletins(target.getControllerServiceBulletins(), toMerge.getControllerServiceBulletins()));

View File

@ -40,6 +40,7 @@ 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;
@ -475,7 +476,7 @@ public class HttpRequestReplicatorImpl implements HttpRequestReplicator {
// create the resource
WebResource resource = client.resource(uri);
if (WebClusterManager.isResponseInterpreted(uri, method)) {
if (new StandardHttpResponseMerger().isResponseInterpreted(uri, method)) {
resource.addFilter(new GZIPContentEncodingFilter(false));
}

View File

@ -17,6 +17,8 @@
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;
@ -25,6 +27,7 @@ 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;
@ -41,10 +44,12 @@ public class WebClusterManagerCoordinator implements ClusterCoordinator {
private final WebClusterManager clusterManager;
private final ClusterManagerProtocolSender protocolSender;
private final DataFlowManagementService dfmService;
public WebClusterManagerCoordinator(final WebClusterManager clusterManager, final ClusterManagerProtocolSender protocolSender) {
public WebClusterManagerCoordinator(final WebClusterManager clusterManager, final ClusterManagerProtocolSender protocolSender, final DataFlowManagementService dfmService) {
this.clusterManager = clusterManager;
this.protocolSender = protocolSender;
this.dfmService = dfmService;
}
@Override
@ -196,6 +201,8 @@ public class WebClusterManagerCoordinator implements ClusterCoordinator {
protocolSender.notifyNodeStatusChange(nodesToNotify, message);
}
dfmService.setNodeIds(getNodeIdentifiers(NodeConnectionState.CONNECTED));
return true;
}
@ -241,6 +248,17 @@ public class WebClusterManagerCoordinator implements ClusterCoordinator {
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;
}
}

View File

@ -20,8 +20,6 @@ import org.apache.nifi.admin.service.AuditService;
import org.apache.nifi.cluster.event.EventManager;
import org.apache.nifi.cluster.firewall.ClusterNodeFirewall;
import org.apache.nifi.cluster.flow.DataFlowManagementService;
import org.apache.nifi.cluster.manager.HttpRequestReplicator;
import org.apache.nifi.cluster.manager.HttpResponseMapper;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import org.apache.nifi.cluster.protocol.impl.ClusterManagerProtocolSenderListener;
import org.apache.nifi.cluster.protocol.impl.ClusterServicesBroadcaster;
@ -61,15 +59,11 @@ public class WebClusterManagerFactoryBean implements FactoryBean, ApplicationCon
*/
return null;
} else if (clusterManager == null) {
final HttpRequestReplicator requestReplicator = applicationContext.getBean("httpRequestReplicator", HttpRequestReplicator.class);
final HttpResponseMapper responseMapper = applicationContext.getBean("httpResponseMapper", HttpResponseMapper.class);
final DataFlowManagementService dataFlowService = applicationContext.getBean("dataFlowManagementService", DataFlowManagementService.class);
final ClusterManagerProtocolSenderListener senderListener = applicationContext.getBean("clusterManagerProtocolSenderListener", ClusterManagerProtocolSenderListener.class);
// create the manager
clusterManager = new WebClusterManager(
requestReplicator,
responseMapper,
dataFlowService,
senderListener,
properties,

View File

@ -25,38 +25,6 @@
http://www.springframework.org/schema/context http://www.springframework.org/schema/context/spring-context-3.1.xsd
http://www.springframework.org/schema/aop http://www.springframework.org/schema/aop/spring-aop-3.1.xsd">
<!-- jersey client -->
<bean id="jersey-client" class="org.apache.nifi.web.util.WebUtils" factory-method="createClient">
<constructor-arg>
<bean class="com.sun.jersey.api.client.config.DefaultClientConfig"/>
</constructor-arg>
<constructor-arg>
<bean class="org.apache.nifi.framework.security.util.SslContextFactory" factory-method="createSslContext">
<constructor-arg ref="nifiProperties"/>
</bean>
</constructor-arg>
</bean>
<!-- http request replicator -->
<bean id="httpRequestReplicator" class="org.apache.nifi.cluster.manager.impl.HttpRequestReplicatorImpl">
<constructor-arg index="0">
<bean factory-bean="nifiProperties" factory-method="getClusterManagerNodeApiRequestThreads"/>
</constructor-arg>
<constructor-arg ref="jersey-client" index="1"/>
<constructor-arg index="2">
<bean factory-bean="nifiProperties" factory-method="getClusterManagerNodeApiConnectionTimeout"/>
</constructor-arg>
<constructor-arg index="3">
<bean factory-bean="nifiProperties" factory-method="getClusterManagerNodeApiReadTimeout"/>
</constructor-arg>
<property name="nodeProtocolScheme">
<bean factory-bean="nifiProperties" factory-method="getClusterProtocolManagerToNodeApiScheme"/>
</property>
</bean>
<!-- http response mapper -->
<bean id="httpResponseMapper" class="org.apache.nifi.cluster.manager.impl.HttpResponseMapperImpl"/>
<!-- cluster flow DAO -->
<bean id="dataFlowDao" class="org.apache.nifi.cluster.flow.impl.DataFlowDaoImpl">
<constructor-arg index="0">

View File

@ -283,6 +283,11 @@ public class TestAbstractHeartbeatMonitor {
public NodeIdentifier getNodeIdentifier(final String uuid) {
return statuses.keySet().stream().filter(p -> p.getId().equals(uuid)).findFirst().orElse(null);
}
@Override
public Map<NodeConnectionState, List<NodeIdentifier>> getConnectionStates() {
return statuses.keySet().stream().collect(Collectors.groupingBy(nodeId -> getConnectionStatus(nodeId).getState()));
}
}
public static class ReportedEvent {

View File

@ -0,0 +1,355 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.coordination.heartbeat;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
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.Properties;
import java.util.Set;
import java.util.UUID;
import java.util.stream.Collectors;
import org.apache.curator.test.TestingServer;
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.reporting.Severity;
import org.apache.nifi.util.NiFiProperties;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
public class TestCuratorHeartbeatMonitor {
private TestingServer zkServer;
private NodeIdentifier nodeId;
private TestFriendlyHeartbeatMonitor monitor;
@Before
public void setup() throws Exception {
zkServer = new TestingServer(true);
zkServer.start();
nodeId = new NodeIdentifier(UUID.randomUUID().toString(), "localhost", 9999, "localhost", 8888, "localhost", null, false);
}
@After
public void clear() throws IOException {
if (zkServer != null) {
zkServer.stop();
zkServer.close();
}
if (monitor != null) {
monitor.stop();
}
}
/**
* Verifies that a node that sends a heartbeat that indicates that it is 'connected' is asked to connect to
* cluster if the cluster coordinator does not know about the node
*
* @throws InterruptedException if interrupted
*/
@Test
public void testNewConnectedHeartbeatFromUnknownNode() throws IOException, InterruptedException {
final List<NodeIdentifier> requestedToConnect = Collections.synchronizedList(new ArrayList<>());
final ClusterCoordinatorAdapter coordinator = new ClusterCoordinatorAdapter() {
@Override
public synchronized void requestNodeConnect(final NodeIdentifier nodeId) {
requestedToConnect.add(nodeId);
}
};
final TestFriendlyHeartbeatMonitor monitor = createMonitor(coordinator);
// Ensure that we request the Unknown Node connect to the cluster
final NodeHeartbeat heartbeat = createHeartbeat(nodeId, NodeConnectionState.CONNECTED);
monitor.addHeartbeat(heartbeat);
monitor.waitForProcessed();
assertEquals(1, requestedToConnect.size());
assertEquals(nodeId, requestedToConnect.get(0));
assertEquals(1, coordinator.getEvents().size());
}
/**
* Verifies that a node that sends a heartbeat that indicates that it is 'connected' if previously
* manually disconnected, will be asked to disconnect from the cluster again.
*
* @throws InterruptedException if interrupted
*/
@Test
public void testHeartbeatFromManuallyDisconnectedNode() throws InterruptedException {
final Set<NodeIdentifier> requestedToConnect = Collections.synchronizedSet(new HashSet<>());
final Set<NodeIdentifier> requestedToDisconnect = Collections.synchronizedSet(new HashSet<>());
final ClusterCoordinatorAdapter adapter = new ClusterCoordinatorAdapter() {
@Override
public synchronized void requestNodeConnect(final NodeIdentifier nodeId) {
super.requestNodeConnect(nodeId);
requestedToConnect.add(nodeId);
}
@Override
public synchronized void requestNodeDisconnect(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode, final String explanation) {
super.requestNodeDisconnect(nodeId, disconnectionCode, explanation);
requestedToDisconnect.add(nodeId);
}
};
final TestFriendlyHeartbeatMonitor monitor = createMonitor(adapter);
adapter.requestNodeDisconnect(nodeId, DisconnectionCode.USER_DISCONNECTED, "Unit Testing");
monitor.addHeartbeat(createHeartbeat(nodeId, NodeConnectionState.CONNECTED));
monitor.waitForProcessed();
assertEquals(1, requestedToDisconnect.size());
assertEquals(nodeId, requestedToDisconnect.iterator().next());
assertTrue(requestedToConnect.isEmpty());
}
@Test
public void testConnectingNodeMarkedConnectedWhenHeartbeatReceived() throws InterruptedException {
final Set<NodeIdentifier> requestedToConnect = Collections.synchronizedSet(new HashSet<>());
final Set<NodeIdentifier> connected = Collections.synchronizedSet(new HashSet<>());
final ClusterCoordinatorAdapter adapter = new ClusterCoordinatorAdapter() {
@Override
public synchronized void requestNodeConnect(final NodeIdentifier nodeId) {
super.requestNodeConnect(nodeId);
requestedToConnect.add(nodeId);
}
@Override
public synchronized void finishNodeConnection(final NodeIdentifier nodeId) {
super.finishNodeConnection(nodeId);
connected.add(nodeId);
}
};
final TestFriendlyHeartbeatMonitor monitor = createMonitor(adapter);
adapter.requestNodeConnect(nodeId); // set state to 'connecting'
requestedToConnect.clear();
monitor.addHeartbeat(createHeartbeat(nodeId, NodeConnectionState.CONNECTED));
monitor.waitForProcessed();
assertEquals(1, connected.size());
assertEquals(nodeId, connected.iterator().next());
assertTrue(requestedToConnect.isEmpty());
}
@Test
public void testDisconnectedHeartbeatOnStartup() throws InterruptedException {
final Set<NodeIdentifier> requestedToConnect = Collections.synchronizedSet(new HashSet<>());
final Set<NodeIdentifier> connected = Collections.synchronizedSet(new HashSet<>());
final Set<NodeIdentifier> disconnected = Collections.synchronizedSet(new HashSet<>());
final ClusterCoordinatorAdapter adapter = new ClusterCoordinatorAdapter() {
@Override
public synchronized void requestNodeConnect(final NodeIdentifier nodeId) {
super.requestNodeConnect(nodeId);
requestedToConnect.add(nodeId);
}
@Override
public synchronized void finishNodeConnection(final NodeIdentifier nodeId) {
super.finishNodeConnection(nodeId);
connected.add(nodeId);
}
@Override
public synchronized void requestNodeDisconnect(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode, final String explanation) {
super.requestNodeDisconnect(nodeId, disconnectionCode, explanation);
disconnected.add(nodeId);
}
};
final TestFriendlyHeartbeatMonitor monitor = createMonitor(adapter);
requestedToConnect.clear();
monitor.addHeartbeat(createHeartbeat(nodeId, DisconnectionCode.NODE_SHUTDOWN));
monitor.waitForProcessed();
assertTrue(connected.isEmpty());
assertTrue(requestedToConnect.isEmpty());
assertTrue(disconnected.isEmpty());
}
private NodeHeartbeat createHeartbeat(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode) {
final NodeConnectionStatus status = new NodeConnectionStatus(disconnectionCode);
return new StandardNodeHeartbeat(nodeId, System.currentTimeMillis(), status, false, 0, 0, 0, 0);
}
private NodeHeartbeat createHeartbeat(final NodeIdentifier nodeId, final NodeConnectionState state) {
final NodeConnectionStatus status = new NodeConnectionStatus(state);
return new StandardNodeHeartbeat(nodeId, System.currentTimeMillis(), status, false, 0, 0, 0, 0);
}
private TestFriendlyHeartbeatMonitor createMonitor(final ClusterCoordinator coordinator) {
monitor = new TestFriendlyHeartbeatMonitor(coordinator, createProperties());
monitor.start();
return monitor;
}
private Properties createProperties() {
final Properties properties = new Properties();
properties.setProperty(NiFiProperties.CLUSTER_PROTOCOL_HEARTBEAT_INTERVAL, "10 ms");
properties.setProperty(NiFiProperties.ZOOKEEPER_CONNECT_STRING, zkServer.getConnectString());
properties.setProperty(NiFiProperties.ZOOKEEPER_CONNECT_TIMEOUT, "3 secs");
properties.setProperty(NiFiProperties.ZOOKEEPER_SESSION_TIMEOUT, "3 secs");
properties.setProperty(NiFiProperties.ZOOKEEPER_ROOT_NODE, "/nifi");
return properties;
}
private static class ClusterCoordinatorAdapter implements ClusterCoordinator {
private final Map<NodeIdentifier, NodeConnectionStatus> statuses = new HashMap<>();
private final List<ReportedEvent> events = new ArrayList<>();
@Override
public synchronized void requestNodeConnect(NodeIdentifier nodeId) {
statuses.put(nodeId, new NodeConnectionStatus(NodeConnectionState.CONNECTING));
}
@Override
public synchronized void finishNodeConnection(NodeIdentifier nodeId) {
statuses.put(nodeId, new NodeConnectionStatus(NodeConnectionState.CONNECTED));
}
@Override
public synchronized void requestNodeDisconnect(NodeIdentifier nodeId, DisconnectionCode disconnectionCode, String explanation) {
statuses.put(nodeId, new NodeConnectionStatus(NodeConnectionState.DISCONNECTED));
}
@Override
public synchronized void disconnectionRequestedByNode(NodeIdentifier nodeId, DisconnectionCode disconnectionCode, String explanation) {
statuses.put(nodeId, new NodeConnectionStatus(NodeConnectionState.DISCONNECTED));
}
@Override
public synchronized NodeConnectionStatus getConnectionStatus(NodeIdentifier nodeId) {
return statuses.get(nodeId);
}
@Override
public synchronized Set<NodeIdentifier> getNodeIdentifiers(NodeConnectionState state) {
return statuses.entrySet().stream().filter(p -> p.getValue().getState() == state).map(p -> p.getKey()).collect(Collectors.toSet());
}
@Override
public synchronized boolean isBlockedByFirewall(String hostname) {
return false;
}
@Override
public synchronized void reportEvent(NodeIdentifier nodeId, Severity severity, String event) {
events.add(new ReportedEvent(nodeId, severity, event));
}
@Override
public synchronized void setPrimaryNode(NodeIdentifier nodeId) {
}
synchronized List<ReportedEvent> getEvents() {
return new ArrayList<>(events);
}
@Override
public NodeIdentifier getNodeIdentifier(final String uuid) {
return statuses.keySet().stream().filter(p -> p.getId().equals(uuid)).findFirst().orElse(null);
}
@Override
public Map<NodeConnectionState, List<NodeIdentifier>> getConnectionStates() {
return null;
}
}
public static class ReportedEvent {
private final NodeIdentifier nodeId;
private final Severity severity;
private final String event;
public ReportedEvent(NodeIdentifier nodeId, Severity severity, String event) {
this.nodeId = nodeId;
this.severity = severity;
this.event = event;
}
public NodeIdentifier getNodeId() {
return nodeId;
}
public Severity getSeverity() {
return severity;
}
public String getEvent() {
return event;
}
}
private static class TestFriendlyHeartbeatMonitor extends CuratorHeartbeatMonitor {
private Map<NodeIdentifier, NodeHeartbeat> heartbeats = new HashMap<>();
private final Object mutex = new Object();
public TestFriendlyHeartbeatMonitor(ClusterCoordinator clusterCoordinator, Properties properties) {
super(clusterCoordinator, properties);
}
@Override
synchronized Map<NodeIdentifier, NodeHeartbeat> fetchHeartbeats() {
return heartbeats;
}
@Override
synchronized void monitorHeartbeats() {
super.monitorHeartbeats();
synchronized (mutex) {
mutex.notify();
}
}
synchronized void addHeartbeat(final NodeHeartbeat heartbeat) {
heartbeats.put(heartbeat.getNodeIdentifier(), heartbeat);
}
@Override
public synchronized void removeHeartbeat(final NodeIdentifier nodeId) {
heartbeats.remove(nodeId);
}
void waitForProcessed() throws InterruptedException {
synchronized (mutex) {
mutex.wait();
}
}
}
}

View File

@ -0,0 +1,64 @@
/*
* 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 static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.junit.Test;
public class TestProcessorEndpointMerger {
@Test
public void testMergeValidationErrors() {
final ProcessorEndpointMerger merger = new ProcessorEndpointMerger();
final Map<String, Set<NodeIdentifier>> validationErrorMap = new HashMap<>();
final NodeIdentifier nodeId1234 = new NodeIdentifier("1234", "localhost", 9000, "localhost", 9001, "localhost", 9002, false);
final List<String> nodeValidationErrors1234 = new ArrayList<>();
nodeValidationErrors1234.add("error 1");
nodeValidationErrors1234.add("error 2");
merger.mergeValidationErrors(validationErrorMap, nodeId1234, nodeValidationErrors1234);
final NodeIdentifier nodeXyz = new NodeIdentifier("xyz", "localhost", 8000, "localhost", 8001, "localhost", 8002, false);
final List<String> nodeValidationErrorsXyz = new ArrayList<>();
nodeValidationErrorsXyz.add("error 1");
merger.mergeValidationErrors(validationErrorMap, nodeXyz, nodeValidationErrorsXyz);
assertEquals(2, validationErrorMap.size());
final Set<NodeIdentifier> idsError1 = validationErrorMap.get("error 1");
assertEquals(2, idsError1.size());
assertTrue(idsError1.contains(nodeId1234));
assertTrue(idsError1.contains(nodeXyz));
final Set<NodeIdentifier> idsError2 = validationErrorMap.get("error 2");
assertEquals(1, idsError2.size());
assertTrue(idsError2.contains(nodeId1234));
}
}

View File

@ -14,7 +14,8 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.manager.impl;
package org.apache.nifi.cluster.coordination.http.endpoints;
import static org.junit.Assert.assertEquals;
@ -26,8 +27,7 @@ import java.util.Locale;
import org.junit.Test;
public class TestWebClusterManager {
public class TestStatusHistoryEndpointMerger {
@Test
public void testNormalizedStatusSnapshotDate() throws ParseException {
final DateFormat df = new SimpleDateFormat("yyyy/MM/dd HH:mm:SS.SSS", Locale.US);
@ -36,17 +36,16 @@ public class TestWebClusterManager {
final Date date3 = df.parse("2014/01/01 00:05:00.000");
final Date date4 = df.parse("2014/01/01 00:05:00.001");
final Date normalized1 = WebClusterManager.normalizeStatusSnapshotDate(date1, 300000);
final Date normalized1 = StatusHistoryEndpointMerger.normalizeStatusSnapshotDate(date1, 300000);
assertEquals(date1, normalized1);
final Date normalized2 = WebClusterManager.normalizeStatusSnapshotDate(date2, 300000);
final Date normalized2 = StatusHistoryEndpointMerger.normalizeStatusSnapshotDate(date2, 300000);
assertEquals(date1, normalized2);
final Date normalized3 = WebClusterManager.normalizeStatusSnapshotDate(date3, 300000);
final Date normalized3 = StatusHistoryEndpointMerger.normalizeStatusSnapshotDate(date3, 300000);
assertEquals(date3, normalized3);
final Date normalized4 = WebClusterManager.normalizeStatusSnapshotDate(date4, 300000);
final Date normalized4 = StatusHistoryEndpointMerger.normalizeStatusSnapshotDate(date4, 300000);
assertEquals(date3, normalized4);
}
}

View File

@ -0,0 +1,126 @@
/*
* 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.replication;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.junit.Test;
import org.mockito.Mockito;
import com.sun.jersey.api.client.ClientResponse;
public class TestResponseUtils {
@Test
public void testFindLongResponseTimes() throws URISyntaxException {
final Map<NodeIdentifier, NodeResponse> responses = new HashMap<>();
final NodeIdentifier id1 = new NodeIdentifier("1", "localhost", 8000, "localhost", 8001, "localhost", 8002, false);
final NodeIdentifier id2 = new NodeIdentifier("2", "localhost", 8200, "localhost", 8201, "localhost", 8202, false);
final NodeIdentifier id3 = new NodeIdentifier("3", "localhost", 8300, "localhost", 8301, "localhost", 8302, false);
final NodeIdentifier id4 = new NodeIdentifier("4", "localhost", 8400, "localhost", 8401, "localhost", 8402, false);
final URI uri = new URI("localhost:8080");
final ClientResponse clientResponse = Mockito.mock(ClientResponse.class);
responses.put(id1, new NodeResponse(id1, "GET", uri, clientResponse, TimeUnit.MILLISECONDS.toNanos(80), "1"));
responses.put(id2, new NodeResponse(id1, "GET", uri, clientResponse, TimeUnit.MILLISECONDS.toNanos(92), "1"));
responses.put(id3, new NodeResponse(id1, "GET", uri, clientResponse, TimeUnit.MILLISECONDS.toNanos(3), "1"));
responses.put(id4, new NodeResponse(id1, "GET", uri, clientResponse, TimeUnit.MILLISECONDS.toNanos(120), "1"));
final AsyncClusterResponse response = new AsyncClusterResponse() {
@Override
public String getRequestIdentifier() {
return "1";
}
@Override
public String getMethod() {
return "GET";
}
@Override
public String getURIPath() {
return null;
}
@Override
public Set<NodeIdentifier> getNodesInvolved() {
return new HashSet<>(responses.keySet());
}
@Override
public Set<NodeIdentifier> getCompletedNodeIdentifiers() {
return getNodesInvolved();
}
@Override
public boolean isComplete() {
return true;
}
@Override
public boolean isOlderThan(long time, TimeUnit timeUnit) {
return true;
}
@Override
public NodeResponse getMergedResponse() {
return null;
}
@Override
public NodeResponse awaitMergedResponse() throws InterruptedException {
return null;
}
@Override
public NodeResponse awaitMergedResponse(long timeout, TimeUnit timeUnit) throws InterruptedException {
return null;
}
@Override
public NodeResponse getNodeResponse(NodeIdentifier nodeId) {
return responses.get(nodeId);
}
@Override
public Set<NodeResponse> getCompletedNodeResponses() {
return new HashSet<>(responses.values());
}
};
Set<NodeIdentifier> slowResponses = ResponseUtils.findLongResponseTimes(response, 1.5D);
assertTrue(slowResponses.isEmpty());
responses.put(id4, new NodeResponse(id1, "GET", uri, clientResponse, TimeUnit.MILLISECONDS.toNanos(2500), "1"));
slowResponses = ResponseUtils.findLongResponseTimes(response, 1.5D);
assertEquals(1, slowResponses.size());
assertEquals(id4, slowResponses.iterator().next());
}
}

View File

@ -0,0 +1,296 @@
/*
* 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.replication;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import java.io.ByteArrayInputStream;
import java.net.SocketTimeoutException;
import java.net.URI;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import javax.ws.rs.HttpMethod;
import org.apache.nifi.cluster.coordination.ClusterCoordinator;
import org.apache.nifi.cluster.flow.DataFlowManagementService;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.manager.exception.IllegalClusterStateException;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.StandardOptimisticLockingManager;
import org.apache.nifi.web.api.entity.Entity;
import org.apache.nifi.web.api.entity.ProcessorEntity;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
import org.mockito.Mockito;
import org.mockito.internal.util.reflection.Whitebox;
import com.sun.jersey.api.client.Client;
import com.sun.jersey.api.client.ClientHandlerException;
import com.sun.jersey.api.client.ClientResponse;
import com.sun.jersey.api.client.ClientResponse.Status;
import com.sun.jersey.api.client.WebResource;
import com.sun.jersey.core.header.InBoundHeaders;
import com.sun.jersey.core.header.OutBoundHeaders;
public class TestThreadPoolRequestReplicator {
@BeforeClass
public static void setupClass() {
System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, "src/test/resources/conf/nifi.properties");
}
/**
* If we replicate a request, whenever we obtain the merged response from the AsyncClusterResponse object,
* the response should no longer be available and should be cleared from internal state. This test is to
* verify that this behavior occurs.
*/
@Test
public void testResponseRemovedWhenCompletedAndFetched() {
withReplicator(replicator -> {
final Set<NodeIdentifier> nodeIds = new HashSet<>();
nodeIds.add(new NodeIdentifier("1", "localhost", 8000, "localhost", 8001, "localhost", 8002, false));
final URI uri = new URI("http://localhost:8080/processors/1");
final Entity entity = new ProcessorEntity();
final AsyncClusterResponse response = replicator.replicate(nodeIds, HttpMethod.GET, uri, entity, new HashMap<>());
// We should get back the same response object
assertTrue(response == replicator.getClusterResponse(response.getRequestIdentifier()));
assertEquals(HttpMethod.GET, response.getMethod());
assertEquals(nodeIds, response.getNodesInvolved());
assertTrue(response == replicator.getClusterResponse(response.getRequestIdentifier()));
final NodeResponse nodeResponse = response.awaitMergedResponse(3, TimeUnit.SECONDS);
assertEquals(8000, nodeResponse.getNodeId().getApiPort());
assertEquals(ClientResponse.Status.OK.getStatusCode(), nodeResponse.getStatus());
assertNull(replicator.getClusterResponse(response.getRequestIdentifier()));
});
}
@Test(timeout = 15000)
public void testLongWaitForResponse() {
withReplicator(replicator -> {
final Set<NodeIdentifier> nodeIds = new HashSet<>();
final NodeIdentifier nodeId = new NodeIdentifier("1", "localhost", 8000, "localhost", 8001, "localhost", 8002, false);
nodeIds.add(nodeId);
final URI uri = new URI("http://localhost:8080/processors/1");
final Entity entity = new ProcessorEntity();
final AsyncClusterResponse response = replicator.replicate(nodeIds, HttpMethod.GET, uri, entity, new HashMap<>());
// We should get back the same response object
assertTrue(response == replicator.getClusterResponse(response.getRequestIdentifier()));
assertFalse(response.isComplete());
final NodeResponse nodeResponse = response.getNodeResponse(nodeId);
assertNull(nodeResponse);
final NodeResponse completedNodeResponse = response.awaitMergedResponse(2, TimeUnit.SECONDS);
assertNotNull(completedNodeResponse);
assertNotNull(completedNodeResponse.getThrowable());
assertEquals(500, completedNodeResponse.getStatus());
assertTrue(response.isComplete());
assertNotNull(response.getMergedResponse());
assertNull(replicator.getClusterResponse(response.getRequestIdentifier()));
} , Status.OK, 1000, new ClientHandlerException(new SocketTimeoutException()));
}
@Test(timeout = 15000)
public void testCompleteOnError() {
withReplicator(replicator -> {
final Set<NodeIdentifier> nodeIds = new HashSet<>();
final NodeIdentifier id1 = new NodeIdentifier("1", "localhost", 8100, "localhost", 8101, "localhost", 8102, false);
final NodeIdentifier id2 = new NodeIdentifier("2", "localhost", 8200, "localhost", 8201, "localhost", 8202, false);
final NodeIdentifier id3 = new NodeIdentifier("3", "localhost", 8300, "localhost", 8301, "localhost", 8302, false);
final NodeIdentifier id4 = new NodeIdentifier("4", "localhost", 8400, "localhost", 8401, "localhost", 8402, false);
nodeIds.add(id1);
nodeIds.add(id2);
nodeIds.add(id3);
nodeIds.add(id4);
final URI uri = new URI("http://localhost:8080/processors/1");
final Entity entity = new ProcessorEntity();
final AsyncClusterResponse response = replicator.replicate(nodeIds, HttpMethod.GET, uri, entity, new HashMap<>());
assertNotNull(response.awaitMergedResponse(1, TimeUnit.SECONDS));
} , null, 0L, new IllegalArgumentException("Exception created for unit test"));
}
@Test(timeout = 15000)
public void testMultipleRequestWithTwoPhaseCommit() {
final Set<NodeIdentifier> nodeIds = new HashSet<>();
nodeIds.add(new NodeIdentifier("1", "localhost", 8100, "localhost", 8101, "localhost", 8102, false));
final ClusterCoordinator coordinator = Mockito.mock(ClusterCoordinator.class);
final AtomicInteger requestCount = new AtomicInteger(0);
final DataFlowManagementService dfmService = Mockito.mock(DataFlowManagementService.class);
final ThreadPoolRequestReplicator replicator = new ThreadPoolRequestReplicator(2, new Client(), coordinator,
"1 sec", "1 sec", null, null, null, new StandardOptimisticLockingManager(), dfmService) {
@Override
protected NodeResponse replicateRequest(final WebResource.Builder resourceBuilder, final NodeIdentifier nodeId, final String method, final URI uri, final String requestId) {
// the resource builder will not expose its headers to us, so we are using Mockito's Whitebox class to extract them.
final OutBoundHeaders headers = (OutBoundHeaders) Whitebox.getInternalState(resourceBuilder, "metadata");
final Object expectsHeader = headers.getFirst(ThreadPoolRequestReplicator.NCM_EXPECTS_HTTP_HEADER);
final int statusCode;
if (requestCount.incrementAndGet() == 1) {
assertEquals(ThreadPoolRequestReplicator.NODE_CONTINUE, expectsHeader);
statusCode = 150;
} else {
assertNull(expectsHeader);
statusCode = Status.OK.getStatusCode();
}
// Return given response from all nodes.
final ClientResponse clientResponse = new ClientResponse(statusCode, new InBoundHeaders(), new ByteArrayInputStream(new byte[0]), null);
return new NodeResponse(nodeId, method, uri, clientResponse, -1L, requestId);
}
};
replicator.start();
try {
final AsyncClusterResponse clusterResponse = replicator.replicate(nodeIds, HttpMethod.POST,
new URI("http://localhost:80/processors/1"), new ProcessorEntity(), new HashMap<>());
clusterResponse.awaitMergedResponse();
// Ensure that we received two requests - the first should contain the X-NcmExpects header; the second should not.
// These assertions are validated above, in the overridden replicateRequest method.
assertEquals(2, requestCount.get());
} catch (final Exception e) {
e.printStackTrace();
Assert.fail(e.toString());
} finally {
replicator.stop();
}
}
@Test(timeout = 15000)
public void testOneNodeRejectsTwoPhaseCommit() {
final Set<NodeIdentifier> nodeIds = new HashSet<>();
nodeIds.add(new NodeIdentifier("1", "localhost", 8100, "localhost", 8101, "localhost", 8102, false));
nodeIds.add(new NodeIdentifier("2", "localhost", 8200, "localhost", 8201, "localhost", 8202, false));
final ClusterCoordinator coordinator = Mockito.mock(ClusterCoordinator.class);
final AtomicInteger requestCount = new AtomicInteger(0);
final DataFlowManagementService dfmService = Mockito.mock(DataFlowManagementService.class);
final ThreadPoolRequestReplicator replicator = new ThreadPoolRequestReplicator(2, new Client(), coordinator,
"1 sec", "1 sec", null, null, null, new StandardOptimisticLockingManager(), dfmService) {
@Override
protected NodeResponse replicateRequest(final WebResource.Builder resourceBuilder, final NodeIdentifier nodeId, final String method, final URI uri, final String requestId) {
// the resource builder will not expose its headers to us, so we are using Mockito's Whitebox class to extract them.
final OutBoundHeaders headers = (OutBoundHeaders) Whitebox.getInternalState(resourceBuilder, "metadata");
final Object expectsHeader = headers.getFirst(ThreadPoolRequestReplicator.NCM_EXPECTS_HTTP_HEADER);
final int requestIndex = requestCount.incrementAndGet();
assertEquals(ThreadPoolRequestReplicator.NODE_CONTINUE, expectsHeader);
if (requestIndex == 1) {
final ClientResponse clientResponse = new ClientResponse(150, new InBoundHeaders(), new ByteArrayInputStream(new byte[0]), null);
return new NodeResponse(nodeId, method, uri, clientResponse, -1L, requestId);
} else {
final IllegalClusterStateException explanation = new IllegalClusterStateException("Intentional Exception for Unit Testing");
return new NodeResponse(nodeId, method, uri, explanation);
}
}
};
replicator.start();
try {
final AsyncClusterResponse clusterResponse = replicator.replicate(nodeIds, HttpMethod.POST,
new URI("http://localhost:80/processors/1"), new ProcessorEntity(), new HashMap<>());
clusterResponse.awaitMergedResponse();
Assert.fail("Expected to get an IllegalClusterStateException but did not");
} catch (final IllegalClusterStateException e) {
// Expected
} catch (final Exception e) {
Assert.fail(e.toString());
} finally {
replicator.stop();
}
}
private void withReplicator(final WithReplicator function) {
withReplicator(function, ClientResponse.Status.OK, 0L, null);
}
private void withReplicator(final WithReplicator function, final Status status, final long delayMillis, final RuntimeException failure) {
final ClusterCoordinator coordinator = Mockito.mock(ClusterCoordinator.class);
final DataFlowManagementService dfmService = Mockito.mock(DataFlowManagementService.class);
final ThreadPoolRequestReplicator replicator = new ThreadPoolRequestReplicator(2, new Client(), coordinator,
"1 sec", "1 sec", null, null, null, new StandardOptimisticLockingManager(), dfmService) {
@Override
protected NodeResponse replicateRequest(final WebResource.Builder resourceBuilder, final NodeIdentifier nodeId, final String method, final URI uri, final String requestId) {
if (delayMillis > 0L) {
try {
Thread.sleep(delayMillis);
} catch (InterruptedException e) {
Assert.fail("Thread Interrupted durating test");
}
}
if (failure != null) {
throw failure;
}
// Return given response from all nodes.
final ClientResponse clientResponse = new ClientResponse(status, new InBoundHeaders(), new ByteArrayInputStream(new byte[0]), null);
return new NodeResponse(nodeId, method, uri, clientResponse, -1L, requestId);
}
};
replicator.start();
try {
function.withReplicator(replicator);
} catch (final Exception e) {
e.printStackTrace();
Assert.fail(e.toString());
} finally {
replicator.stop();
}
}
private interface WithReplicator {
void withReplicator(ThreadPoolRequestReplicator replicator) throws Exception;
}
}

View File

@ -44,6 +44,7 @@ import org.apache.nifi.cluster.manager.testutils.HttpResponse;
import org.apache.nifi.cluster.manager.testutils.HttpResponseAction;
import org.apache.nifi.cluster.manager.testutils.HttpServer;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.util.NiFiProperties;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@ -69,6 +70,7 @@ public class HttpRequestReplicatorImplTest {
@Before
public void setUp() throws IOException, URISyntaxException {
System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, getClass().getResource("/conf/nifi.properties").getFile());
executorThreadCount = 5;
serverThreadCount = 3;

View File

@ -0,0 +1,127 @@
# 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.
# Core Properties #
nifi.version=nifi-test 3.0.0
nifi.flow.configuration.file=./target/flow.xml.gz
nifi.flow.configuration.archive.dir=./target/archive/
nifi.flowcontroller.autoResumeState=true
nifi.flowcontroller.graceful.shutdown.period=10 sec
nifi.flowservice.writedelay.interval=2 sec
nifi.administrative.yield.duration=30 sec
nifi.reporting.task.configuration.file=./target/reporting-tasks.xml
nifi.controller.service.configuration.file=./target/controller-services.xml
nifi.templates.directory=./target/templates
nifi.ui.banner.text=UI Banner Text
nifi.ui.autorefresh.interval=30 sec
nifi.nar.library.directory=./target/lib
nifi.nar.working.directory=./target/work/nar/
# H2 Settings
nifi.database.directory=./database_repository
nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE
# FlowFile Repository
nifi.flowfile.repository.directory=./target/test-repo
nifi.flowfile.repository.partitions=1
nifi.flowfile.repository.checkpoint.interval=2 mins
nifi.queue.swap.threshold=20000
nifi.swap.storage.directory=./target/test-repo/swap
nifi.swap.in.period=5 sec
nifi.swap.in.threads=1
nifi.swap.out.period=5 sec
nifi.swap.out.threads=4
# Content Repository
nifi.content.claim.max.appendable.size=10 MB
nifi.content.claim.max.flow.files=100
nifi.content.repository.directory.default=./target/content_repository
# Provenance Repository Properties
nifi.provenance.repository.storage.directory=./target/provenance_repository
nifi.provenance.repository.max.storage.time=24 hours
nifi.provenance.repository.max.storage.size=1 GB
nifi.provenance.repository.rollover.time=30 secs
nifi.provenance.repository.rollover.size=100 MB
# Site to Site properties
nifi.remote.input.socket.port=9990
nifi.remote.input.secure=true
# web properties #
nifi.web.war.directory=./target/lib
nifi.web.http.host=
nifi.web.http.port=8080
nifi.web.https.host=
nifi.web.https.port=
nifi.web.jetty.working.directory=./target/work/jetty
# security properties #
nifi.sensitive.props.key=key
nifi.sensitive.props.algorithm=PBEWITHMD5AND256BITAES-CBC-OPENSSL
nifi.sensitive.props.provider=BC
nifi.security.keystore=
nifi.security.keystoreType=
nifi.security.keystorePasswd=
nifi.security.keyPasswd=
nifi.security.truststore=
nifi.security.truststoreType=
nifi.security.truststorePasswd=
nifi.security.needClientAuth=
nifi.security.authorizedUsers.file=./target/conf/authorized-users.xml
nifi.security.user.credential.cache.duration=24 hours
nifi.security.user.authority.provider=nifi.authorization.FileAuthorizationProvider
nifi.security.support.new.account.requests=
nifi.security.default.user.roles=
# cluster common properties (cluster manager and nodes must have same values) #
nifi.cluster.protocol.heartbeat.interval=5 sec
nifi.cluster.protocol.is.secure=false
nifi.cluster.protocol.socket.timeout=30 sec
nifi.cluster.protocol.connection.handshake.timeout=45 sec
# if multicast is used, then nifi.cluster.protocol.multicast.xxx properties must be configured #
nifi.cluster.protocol.use.multicast=false
nifi.cluster.protocol.multicast.address=
nifi.cluster.protocol.multicast.port=
nifi.cluster.protocol.multicast.service.broadcast.delay=500 ms
nifi.cluster.protocol.multicast.service.locator.attempts=3
nifi.cluster.protocol.multicast.service.locator.attempts.delay=1 sec
# cluster node properties (only configure for cluster nodes) #
nifi.cluster.is.node=false
nifi.cluster.node.address=
nifi.cluster.node.protocol.port=
nifi.cluster.node.protocol.threads=2
# if multicast is not used, nifi.cluster.node.unicast.xxx must have same values as nifi.cluster.manager.xxx #
nifi.cluster.node.unicast.manager.address=
nifi.cluster.node.unicast.manager.protocol.port=
nifi.cluster.node.unicast.manager.authority.provider.port=
# cluster manager properties (only configure for cluster manager) #
nifi.cluster.is.manager=false
nifi.cluster.manager.address=
nifi.cluster.manager.protocol.port=
nifi.cluster.manager.authority.provider.port=
nifi.cluster.manager.authority.provider.threads=10
nifi.cluster.manager.node.firewall.file=
nifi.cluster.manager.node.event.history.size=10
nifi.cluster.manager.node.api.connection.timeout=30 sec
nifi.cluster.manager.node.api.read.timeout=30 sec
nifi.cluster.manager.node.api.request.threads=10
nifi.cluster.manager.flow.retrieval.delay=5 sec
nifi.cluster.manager.protocol.threads=10
nifi.cluster.manager.safemode.duration=0 sec

View File

@ -20,6 +20,7 @@ import java.util.Collection;
import java.util.Set;
import org.apache.nifi.annotation.lifecycle.OnAdded;
import org.apache.nifi.controller.ConfiguredComponent;
import org.apache.nifi.controller.ControllerServiceLookup;
/**
@ -109,7 +110,7 @@ public interface ControllerServiceProvider extends ControllerServiceLookup {
*
* @param serviceNode the node
*/
void unscheduleReferencingComponents(ControllerServiceNode serviceNode);
Set<ConfiguredComponent> unscheduleReferencingComponents(ControllerServiceNode serviceNode);
/**
* Verifies that all Controller Services referencing the provided Controller
@ -130,7 +131,7 @@ public interface ControllerServiceProvider extends ControllerServiceLookup {
*
* @param serviceNode the node
*/
void disableReferencingServices(ControllerServiceNode serviceNode);
Set<ConfiguredComponent> disableReferencingServices(ControllerServiceNode serviceNode);
/**
* Verifies that all Controller Services referencing the provided
@ -149,8 +150,10 @@ public interface ControllerServiceProvider extends ControllerServiceLookup {
* Service A and B will both be enabled.
*
* @param serviceNode the node
*
* @return the set of all components that were updated as a result of this action
*/
void enableReferencingServices(ControllerServiceNode serviceNode);
Set<ConfiguredComponent> enableReferencingServices(ControllerServiceNode serviceNode);
/**
* Verifies that all enabled Processors referencing the ControllerService
@ -172,5 +175,5 @@ public interface ControllerServiceProvider extends ControllerServiceLookup {
*
* @param serviceNode the node
*/
void scheduleReferencingComponents(ControllerServiceNode serviceNode);
Set<ConfiguredComponent> scheduleReferencingComponents(ControllerServiceNode serviceNode);
}

View File

@ -83,7 +83,7 @@ import org.apache.nifi.connectable.Position;
import org.apache.nifi.connectable.Size;
import org.apache.nifi.connectable.StandardConnection;
import org.apache.nifi.controller.cluster.Heartbeater;
import org.apache.nifi.controller.cluster.ZooKeeperHeartbeater;
import org.apache.nifi.controller.cluster.ClusterProtocolHeartbeater;
import org.apache.nifi.controller.exception.CommunicationsException;
import org.apache.nifi.controller.exception.ComponentLifeCycleException;
import org.apache.nifi.controller.exception.ProcessorInstantiationException;
@ -534,7 +534,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
if (configuredForClustering) {
leaderElectionManager = new CuratorLeaderElectionManager(4);
heartbeater = new ZooKeeperHeartbeater(protocolSender, properties);
heartbeater = new ClusterProtocolHeartbeater(protocolSender, properties);
} else {
leaderElectionManager = null;
heartbeater = null;
@ -1273,6 +1273,11 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
private void sendShutdownNotification() {
// Generate a heartbeat message and publish it, indicating that we are shutting down
final HeartbeatMessage heartbeatMsg = createHeartbeatMessage();
if (heartbeatMsg == null) {
LOG.warn("Cannot sent Shutdown Notification Message because node's identifier is not known at this time");
return;
}
final Heartbeat heartbeat = heartbeatMsg.getHeartbeat();
final byte[] hbPayload = heartbeatMsg.getHeartbeat().getPayload();
final NodeConnectionStatus connectionStatus = new NodeConnectionStatus(DisconnectionCode.NODE_SHUTDOWN);
@ -2540,6 +2545,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
status.setProcessingNanos(0);
status.setInvocations(0);
status.setAverageLineageDuration(0L);
status.setFlowFilesRemoved(0);
} else {
final int processedCount = entry.getFlowFilesOut();
final long numProcessedBytes = entry.getContentSizeOut();
@ -2566,6 +2572,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
status.setBytesReceived(entry.getBytesReceived());
status.setFlowFilesSent(entry.getFlowFilesSent());
status.setBytesSent(entry.getBytesSent());
status.setFlowFilesRemoved(entry.getFlowFilesRemoved());
}
// determine the run status and get any validation errors... must check
@ -2835,23 +2842,23 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
}
@Override
public void disableReferencingServices(final ControllerServiceNode serviceNode) {
controllerServiceProvider.disableReferencingServices(serviceNode);
public Set<ConfiguredComponent> disableReferencingServices(final ControllerServiceNode serviceNode) {
return controllerServiceProvider.disableReferencingServices(serviceNode);
}
@Override
public void enableReferencingServices(final ControllerServiceNode serviceNode) {
controllerServiceProvider.enableReferencingServices(serviceNode);
public Set<ConfiguredComponent> enableReferencingServices(final ControllerServiceNode serviceNode) {
return controllerServiceProvider.enableReferencingServices(serviceNode);
}
@Override
public void scheduleReferencingComponents(final ControllerServiceNode serviceNode) {
controllerServiceProvider.scheduleReferencingComponents(serviceNode);
public Set<ConfiguredComponent> scheduleReferencingComponents(final ControllerServiceNode serviceNode) {
return controllerServiceProvider.scheduleReferencingComponents(serviceNode);
}
@Override
public void unscheduleReferencingComponents(final ControllerServiceNode serviceNode) {
controllerServiceProvider.unscheduleReferencingComponents(serviceNode);
public Set<ConfiguredComponent> unscheduleReferencingComponents(final ControllerServiceNode serviceNode) {
return controllerServiceProvider.unscheduleReferencingComponents(serviceNode);
}
@Override
@ -3744,7 +3751,13 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
hbPayload.setTotalFlowFileBytes(queueSize.getByteCount());
// create heartbeat message
final Heartbeat heartbeat = new Heartbeat(getNodeId(), bean.isPrimary(), bean.getConnectionStatus(), hbPayload.marshal());
final NodeIdentifier nodeId = getNodeId();
if (nodeId == null) {
LOG.warn("Cannot create Heartbeat Message because node's identifier is not known at this time");
return null;
}
final Heartbeat heartbeat = new Heartbeat(nodeId, bean.isPrimary(), bean.getConnectionStatus(), hbPayload.marshal());
final HeartbeatMessage message = new HeartbeatMessage();
message.setHeartbeat(heartbeat);

View File

@ -17,6 +17,8 @@
package org.apache.nifi.controller;
import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
@ -203,6 +205,18 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
}
@Override
public void archiveFlow() throws IOException {
writeLock.lock();
try {
final File archiveFile = dao.createArchiveFile();
try (final OutputStream out = new FileOutputStream(archiveFile)) {
dao.load(out, true);
}
} finally {
writeLock.unlock();
}
}
@Override
public void saveFlowChanges() throws IOException {

View File

@ -16,13 +16,15 @@
*/
package org.apache.nifi.controller;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.Set;
import java.util.HashMap;
import java.util.Map;
import javax.xml.bind.annotation.XmlRootElement;
import org.apache.nifi.web.Revision;
/**
* Represents a data flow snippet.
*/
@ -33,14 +35,14 @@ public class StandardSnippet implements Snippet {
private String parentGroupId;
private Boolean linked;
private final Set<String> processGroups = new HashSet<>();
private final Set<String> remoteProcessGroups = new HashSet<>();
private final Set<String> processors = new HashSet<>();
private final Set<String> inputPorts = new HashSet<>();
private final Set<String> outputPorts = new HashSet<>();
private final Set<String> connections = new HashSet<>();
private final Set<String> labels = new HashSet<>();
private final Set<String> funnels = new HashSet<>();
private Map<String, Revision> processGroups = new HashMap<>();
private Map<String, Revision> remoteProcessGroups = new HashMap<>();
private Map<String, Revision> processors = new HashMap<>();
private Map<String, Revision> inputPorts = new HashMap<>();
private Map<String, Revision> outputPorts = new HashMap<>();
private Map<String, Revision> connections = new HashMap<>();
private Map<String, Revision> labels = new HashMap<>();
private Map<String, Revision> funnels = new HashMap<>();
@Override
public String getId() {
@ -74,75 +76,75 @@ public class StandardSnippet implements Snippet {
}
@Override
public Set<String> getConnections() {
return Collections.unmodifiableSet(connections);
public Map<String, Revision> getConnections() {
return Collections.unmodifiableMap(connections);
}
public void addConnections(Collection<String> ids) {
connections.addAll(ids);
public void addConnections(Map<String, Revision> ids) {
connections.putAll(ids);
}
@Override
public Set<String> getFunnels() {
return Collections.unmodifiableSet(funnels);
public Map<String, Revision> getFunnels() {
return Collections.unmodifiableMap(funnels);
}
public void addFunnels(Collection<String> ids) {
funnels.addAll(ids);
public void addFunnels(Map<String, Revision> ids) {
funnels.putAll(ids);
}
@Override
public Set<String> getInputPorts() {
return Collections.unmodifiableSet(inputPorts);
public Map<String, Revision> getInputPorts() {
return Collections.unmodifiableMap(inputPorts);
}
public void addInputPorts(Collection<String> ids) {
inputPorts.addAll(ids);
public void addInputPorts(Map<String, Revision> ids) {
inputPorts.putAll(ids);
}
@Override
public Set<String> getOutputPorts() {
return Collections.unmodifiableSet(outputPorts);
public Map<String, Revision> getOutputPorts() {
return Collections.unmodifiableMap(outputPorts);
}
public void addOutputPorts(Collection<String> ids) {
outputPorts.addAll(ids);
public void addOutputPorts(Map<String, Revision> ids) {
outputPorts.putAll(ids);
}
@Override
public Set<String> getLabels() {
return Collections.unmodifiableSet(labels);
public Map<String, Revision> getLabels() {
return Collections.unmodifiableMap(labels);
}
public void addLabels(Collection<String> ids) {
labels.addAll(ids);
public void addLabels(Map<String, Revision> ids) {
labels.putAll(ids);
}
@Override
public Set<String> getProcessGroups() {
return Collections.unmodifiableSet(processGroups);
public Map<String, Revision> getProcessGroups() {
return Collections.unmodifiableMap(processGroups);
}
public void addProcessGroups(Collection<String> ids) {
processGroups.addAll(ids);
public void addProcessGroups(Map<String, Revision> ids) {
processGroups.putAll(ids);
}
@Override
public Set<String> getProcessors() {
return Collections.unmodifiableSet(processors);
public Map<String, Revision> getProcessors() {
return Collections.unmodifiableMap(processors);
}
public void addProcessors(Collection<String> ids) {
processors.addAll(ids);
public void addProcessors(Map<String, Revision> ids) {
processors.putAll(ids);
}
@Override
public Set<String> getRemoteProcessGroups() {
return Collections.unmodifiableSet(remoteProcessGroups);
public Map<String, Revision> getRemoteProcessGroups() {
return Collections.unmodifiableMap(remoteProcessGroups);
}
public void addRemoteProcessGroups(Collection<String> ids) {
remoteProcessGroups.addAll(ids);
public void addRemoteProcessGroups(Map<String, Revision> ids) {
remoteProcessGroups.putAll(ids);
}
@Override

View File

@ -38,8 +38,8 @@ import org.slf4j.LoggerFactory;
* that this node is part of the cluster. However, once the Cluster Coordinator is known, heartbeats are
* sent directly to the Cluster Coordinator.
*/
public class ZooKeeperHeartbeater implements Heartbeater {
private static final Logger logger = LoggerFactory.getLogger(ZooKeeperHeartbeater.class);
public class ClusterProtocolHeartbeater implements Heartbeater {
private static final Logger logger = LoggerFactory.getLogger(ClusterProtocolHeartbeater.class);
private final NodeProtocolSender protocolSender;
private final CuratorFramework curatorClient;
@ -49,7 +49,7 @@ public class ZooKeeperHeartbeater implements Heartbeater {
private volatile String coordinatorAddress;
public ZooKeeperHeartbeater(final NodeProtocolSender protocolSender, final Properties properties) {
public ClusterProtocolHeartbeater(final NodeProtocolSender protocolSender, final Properties properties) {
this.protocolSender = protocolSender;
final RetryPolicy retryPolicy = new RetryForever(5000);
@ -86,6 +86,7 @@ public class ZooKeeperHeartbeater implements Heartbeater {
}
}
@Override
public synchronized void send(final HeartbeatMessage heartbeatMessage) throws IOException {
final String heartbeatAddress = getHeartbeatAddress();

View File

@ -17,15 +17,12 @@
package org.apache.nifi.controller.cluster;
import java.util.List;
import java.util.Properties;
import java.util.concurrent.TimeUnit;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.util.NiFiProperties;
import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.common.PathUtils;
import org.apache.zookeeper.data.ACL;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -36,14 +33,12 @@ public class ZooKeeperClientConfig {
private final int sessionTimeoutMillis;
private final int connectionTimeoutMillis;
private final String rootPath;
private final List<ACL> acls;
private ZooKeeperClientConfig(String connectString, int sessionTimeoutMillis, int connectionTimeoutMillis, String rootPath, List<ACL> acls) {
private ZooKeeperClientConfig(String connectString, int sessionTimeoutMillis, int connectionTimeoutMillis, String rootPath) {
this.connectString = connectString;
this.sessionTimeoutMillis = sessionTimeoutMillis;
this.connectionTimeoutMillis = connectionTimeoutMillis;
this.rootPath = rootPath.endsWith("/") ? rootPath.substring(0, rootPath.length() - 1) : rootPath;
this.acls = acls;
}
public String getConnectString() {
@ -62,10 +57,6 @@ public class ZooKeeperClientConfig {
return rootPath;
}
public List<ACL> getACLs() {
return acls;
}
public String resolvePath(final String path) {
if (path.startsWith("/")) {
return rootPath + path;
@ -83,18 +74,6 @@ public class ZooKeeperClientConfig {
final long sessionTimeoutMs = getTimePeriod(properties, NiFiProperties.ZOOKEEPER_SESSION_TIMEOUT, NiFiProperties.DEFAULT_ZOOKEEPER_SESSION_TIMEOUT);
final long connectionTimeoutMs = getTimePeriod(properties, NiFiProperties.ZOOKEEPER_CONNECT_TIMEOUT, NiFiProperties.DEFAULT_ZOOKEEPER_CONNECT_TIMEOUT);
final String rootPath = properties.getProperty(NiFiProperties.ZOOKEEPER_ROOT_NODE, NiFiProperties.DEFAULT_ZOOKEEPER_ROOT_NODE);
final String accessControl = properties.getProperty(NiFiProperties.ZOOKEEPER_ACCESS_CONTROL);
final List<ACL> acls;
if (accessControl == null || accessControl.trim().isEmpty()) {
acls = null;
} else if (accessControl.equalsIgnoreCase("Open")) {
acls = Ids.OPEN_ACL_UNSAFE;
} else if (accessControl.equalsIgnoreCase("CreatorOnly")) {
acls = Ids.CREATOR_ALL_ACL;
} else {
acls = null;
}
try {
PathUtils.validatePath(rootPath);
@ -102,7 +81,7 @@ public class ZooKeeperClientConfig {
throw new IllegalArgumentException("The '" + NiFiProperties.ZOOKEEPER_ROOT_NODE + "' property in nifi.properties is set to an illegal value: " + rootPath);
}
return new ZooKeeperClientConfig(connectString, (int) sessionTimeoutMs, (int) connectionTimeoutMs, rootPath, acls);
return new ZooKeeperClientConfig(connectString, (int) sessionTimeoutMs, (int) connectionTimeoutMs, rootPath);
}
private static int getTimePeriod(final Properties properties, final String propertyName, final String defaultValue) {

View File

@ -207,39 +207,46 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
}
@Override
public void disableReferencingServices(final ControllerServiceNode serviceNode) {
public Set<ConfiguredComponent> disableReferencingServices(final ControllerServiceNode serviceNode) {
// Get a list of all Controller Services that need to be disabled, in the order that they need to be
// disabled.
final List<ControllerServiceNode> toDisable = findRecursiveReferences(serviceNode, ControllerServiceNode.class);
final Set<ControllerServiceNode> serviceSet = new HashSet<>(toDisable);
final Set<ConfiguredComponent> updated = new HashSet<>();
for (final ControllerServiceNode nodeToDisable : toDisable) {
if (nodeToDisable.isActive()) {
nodeToDisable.verifyCanDisable(serviceSet);
updated.add(nodeToDisable);
}
}
Collections.reverse(toDisable);
processScheduler.disableControllerServices(toDisable);
return updated;
}
@Override
public void scheduleReferencingComponents(final ControllerServiceNode serviceNode) {
public Set<ConfiguredComponent> scheduleReferencingComponents(final ControllerServiceNode serviceNode) {
// find all of the schedulable components (processors, reporting tasks) that refer to this Controller Service,
// or a service that references this controller service, etc.
final List<ProcessorNode> processors = findRecursiveReferences(serviceNode, ProcessorNode.class);
final List<ReportingTaskNode> reportingTasks = findRecursiveReferences(serviceNode, ReportingTaskNode.class);
final Set<ConfiguredComponent> updated = new HashSet<>();
// verify that we can start all components (that are not disabled) before doing anything
for (final ProcessorNode node : processors) {
if (node.getScheduledState() != ScheduledState.DISABLED) {
node.verifyCanStart();
updated.add(node);
}
}
for (final ReportingTaskNode node : reportingTasks) {
if (node.getScheduledState() != ScheduledState.DISABLED) {
node.verifyCanStart();
updated.add(node);
}
}
@ -247,22 +254,28 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
for (final ProcessorNode node : processors) {
if (node.getScheduledState() != ScheduledState.DISABLED) {
node.getProcessGroup().startProcessor(node);
updated.add(node);
}
}
for (final ReportingTaskNode node : reportingTasks) {
if (node.getScheduledState() != ScheduledState.DISABLED) {
processScheduler.schedule(node);
updated.add(node);
}
}
return updated;
}
@Override
public void unscheduleReferencingComponents(final ControllerServiceNode serviceNode) {
public Set<ConfiguredComponent> unscheduleReferencingComponents(final ControllerServiceNode serviceNode) {
// find all of the schedulable components (processors, reporting tasks) that refer to this Controller Service,
// or a service that references this controller service, etc.
final List<ProcessorNode> processors = findRecursiveReferences(serviceNode, ProcessorNode.class);
final List<ReportingTaskNode> reportingTasks = findRecursiveReferences(serviceNode, ReportingTaskNode.class);
final Set<ConfiguredComponent> updated = new HashSet<>();
// verify that we can stop all components (that are running) before doing anything
for (final ProcessorNode node : processors) {
if (node.getScheduledState() == ScheduledState.RUNNING) {
@ -279,13 +292,17 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
for (final ProcessorNode node : processors) {
if (node.getScheduledState() == ScheduledState.RUNNING) {
node.getProcessGroup().stopProcessor(node);
updated.add(node);
}
}
for (final ReportingTaskNode node : reportingTasks) {
if (node.getScheduledState() == ScheduledState.RUNNING) {
processScheduler.unschedule(node);
updated.add(node);
}
}
return updated;
}
@Override
@ -541,16 +558,18 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
}
@Override
public void enableReferencingServices(final ControllerServiceNode serviceNode) {
public Set<ConfiguredComponent> enableReferencingServices(final ControllerServiceNode serviceNode) {
final List<ControllerServiceNode> recursiveReferences = findRecursiveReferences(serviceNode, ControllerServiceNode.class);
enableReferencingServices(serviceNode, recursiveReferences);
return enableReferencingServices(serviceNode, recursiveReferences);
}
private void enableReferencingServices(final ControllerServiceNode serviceNode, final List<ControllerServiceNode> recursiveReferences) {
private Set<ConfiguredComponent> enableReferencingServices(final ControllerServiceNode serviceNode, final List<ControllerServiceNode> recursiveReferences) {
if (!serviceNode.isActive()) {
serviceNode.verifyCanEnable(new HashSet<>(recursiveReferences));
}
final Set<ConfiguredComponent> updated = new HashSet<>();
final Set<ControllerServiceNode> ifEnabled = new HashSet<>();
for (final ControllerServiceNode nodeToEnable : recursiveReferences) {
if (!nodeToEnable.isActive()) {
@ -562,8 +581,11 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
for (final ControllerServiceNode nodeToEnable : recursiveReferences) {
if (!nodeToEnable.isActive()) {
enableControllerService(nodeToEnable);
updated.add(nodeToEnable);
}
}
return updated;
}
@Override

View File

@ -26,72 +26,42 @@ public enum ConnectionStatusDescriptor {
"Bytes In (5 mins)",
"The cumulative size of all FlowFiles that were transferred to this Connection in the past 5 minutes",
Formatter.DATA_SIZE,
new ValueMapper<ConnectionStatus>() {
@Override
public Long getValue(final ConnectionStatus status) {
return status.getInputBytes();
}
})),
s -> s.getInputBytes())),
INPUT_COUNT(new StandardMetricDescriptor<ConnectionStatus>(
"inputCount",
"FlowFiles In (5 mins)",
"The number of FlowFiles that were transferred to this Connection in the past 5 minutes",
Formatter.COUNT,
new ValueMapper<ConnectionStatus>() {
@Override
public Long getValue(final ConnectionStatus status) {
return Long.valueOf(status.getInputCount());
}
})),
s -> Long.valueOf(s.getInputCount()))),
OUTPUT_BYTES(new StandardMetricDescriptor<ConnectionStatus>(
"outputBytes",
"Bytes Out (5 mins)",
"The cumulative size of all FlowFiles that were pulled from this Connection in the past 5 minutes",
Formatter.DATA_SIZE,
new ValueMapper<ConnectionStatus>() {
@Override
public Long getValue(final ConnectionStatus status) {
return status.getOutputBytes();
}
})),
s -> s.getOutputBytes())),
OUTPUT_COUNT(new StandardMetricDescriptor<ConnectionStatus>(
"outputCount",
"FlowFiles Out (5 mins)",
"The number of FlowFiles that were pulled from this Connection in the past 5 minutes",
Formatter.COUNT,
new ValueMapper<ConnectionStatus>() {
@Override
public Long getValue(final ConnectionStatus status) {
return Long.valueOf(status.getOutputCount());
}
})),
s -> Long.valueOf(s.getOutputCount()))),
QUEUED_BYTES(new StandardMetricDescriptor<ConnectionStatus>(
"queuedBytes",
"Queued Bytes",
"The number of Bytes queued in this Connection",
Formatter.DATA_SIZE,
new ValueMapper<ConnectionStatus>() {
@Override
public Long getValue(final ConnectionStatus status) {
return status.getQueuedBytes();
}
})),
s -> s.getQueuedBytes())),
QUEUED_COUNT(new StandardMetricDescriptor<ConnectionStatus>(
"queuedCount",
"Queued Count",
"The number of FlowFiles queued in this Connection",
Formatter.COUNT,
new ValueMapper<ConnectionStatus>() {
@Override
public Long getValue(final ConnectionStatus status) {
return Long.valueOf(status.getQueuedCount());
}
}));
s -> Long.valueOf(s.getQueuedCount())));
private MetricDescriptor<ConnectionStatus> descriptor;

View File

@ -25,92 +25,66 @@ import org.apache.nifi.controller.status.history.MetricDescriptor.Formatter;
public enum ProcessGroupStatusDescriptor {
BYTES_READ(new StandardMetricDescriptor<ProcessGroupStatus>("bytesRead", "Bytes Read (5 mins)",
"The total number of bytes read from Content Repository by Processors in this Process Group in the past 5 minutes", Formatter.DATA_SIZE, new ValueMapper<ProcessGroupStatus>() {
@Override
public Long getValue(final ProcessGroupStatus status) {
return status.getBytesRead();
}
})),
BYTES_READ(new StandardMetricDescriptor<ProcessGroupStatus>(
"bytesRead",
"Bytes Read (5 mins)",
"The total number of bytes read from Content Repository by Processors in this Process Group in the past 5 minutes",
Formatter.DATA_SIZE,
s -> s.getBytesRead())),
BYTES_WRITTEN(new StandardMetricDescriptor<ProcessGroupStatus>("bytesWritten", "Bytes Written (5 mins)",
"The total number of bytes written to Content Repository by Processors in this Process Group in the past 5 minutes", Formatter.DATA_SIZE, new ValueMapper<ProcessGroupStatus>() {
@Override
public Long getValue(final ProcessGroupStatus status) {
return status.getBytesWritten();
}
})),
BYTES_WRITTEN(new StandardMetricDescriptor<ProcessGroupStatus>("bytesWritten",
"Bytes Written (5 mins)",
"The total number of bytes written to Content Repository by Processors in this Process Group in the past 5 minutes",
Formatter.DATA_SIZE,
s -> s.getBytesWritten())),
BYTES_TRANSFERRED(new StandardMetricDescriptor<ProcessGroupStatus>("bytesTransferred", "Bytes Transferred (5 mins)",
BYTES_TRANSFERRED(new StandardMetricDescriptor<ProcessGroupStatus>("bytesTransferred",
"Bytes Transferred (5 mins)",
"The total number of bytes read from or written to Content Repository by Processors in this Process Group in the past 5 minutes",
Formatter.DATA_SIZE, new ValueMapper<ProcessGroupStatus>() {
@Override
public Long getValue(final ProcessGroupStatus status) {
return status.getBytesRead() + status.getBytesWritten();
}
})),
Formatter.DATA_SIZE,
s -> s.getBytesRead() + s.getBytesWritten())),
INPUT_BYTES(new StandardMetricDescriptor<ProcessGroupStatus>("inputBytes", "Bytes In (5 mins)",
INPUT_BYTES(new StandardMetricDescriptor<ProcessGroupStatus>("inputBytes",
"Bytes In (5 mins)",
"The cumulative size of all FlowFiles that have entered this Process Group via its Input Ports in the past 5 minutes",
Formatter.DATA_SIZE, new ValueMapper<ProcessGroupStatus>() {
@Override
public Long getValue(final ProcessGroupStatus status) {
return status.getInputContentSize();
}
})),
Formatter.DATA_SIZE,
s -> s.getInputContentSize())),
INPUT_COUNT(new StandardMetricDescriptor<ProcessGroupStatus>("inputCount", "FlowFiles In (5 mins)",
INPUT_COUNT(new StandardMetricDescriptor<ProcessGroupStatus>("inputCount",
"FlowFiles In (5 mins)",
"The number of FlowFiles that have entered this Process Group via its Input Ports in the past 5 minutes",
Formatter.COUNT, new ValueMapper<ProcessGroupStatus>() {
@Override
public Long getValue(final ProcessGroupStatus status) {
return status.getInputCount().longValue();
}
})),
Formatter.COUNT,
s -> s.getInputCount().longValue())),
OUTPUT_BYTES(new StandardMetricDescriptor<ProcessGroupStatus>("outputBytes", "Bytes Out (5 mins)",
OUTPUT_BYTES(new StandardMetricDescriptor<ProcessGroupStatus>("outputBytes",
"Bytes Out (5 mins)",
"The cumulative size of all FlowFiles that have exited this Process Group via its Output Ports in the past 5 minutes",
Formatter.DATA_SIZE, new ValueMapper<ProcessGroupStatus>() {
@Override
public Long getValue(final ProcessGroupStatus status) {
return status.getOutputContentSize();
}
})),
Formatter.DATA_SIZE,
s -> s.getOutputContentSize())),
OUTPUT_COUNT(new StandardMetricDescriptor<ProcessGroupStatus>("outputCount", "FlowFiles Out (5 mins)",
OUTPUT_COUNT(new StandardMetricDescriptor<ProcessGroupStatus>("outputCount",
"FlowFiles Out (5 mins)",
"The number of FlowFiles that have exited this Process Group via its Output Ports in the past 5 minutes",
Formatter.COUNT, new ValueMapper<ProcessGroupStatus>() {
@Override
public Long getValue(final ProcessGroupStatus status) {
return status.getOutputCount().longValue();
}
})),
Formatter.COUNT,
s -> s.getOutputCount().longValue())),
QUEUED_BYTES(new StandardMetricDescriptor<ProcessGroupStatus>("queuedBytes", "Queued Bytes",
QUEUED_BYTES(new StandardMetricDescriptor<ProcessGroupStatus>("queuedBytes",
"Queued Bytes",
"The cumulative size of all FlowFiles queued in all Connections of this Process Group",
Formatter.DATA_SIZE, new ValueMapper<ProcessGroupStatus>() {
@Override
public Long getValue(final ProcessGroupStatus status) {
return status.getQueuedContentSize();
}
})),
Formatter.DATA_SIZE,
s -> s.getQueuedContentSize())),
QUEUED_COUNT(new StandardMetricDescriptor<ProcessGroupStatus>("queuedCount", "Queued Count",
"The number of FlowFiles queued in all Connections of this Process Group", Formatter.COUNT, new ValueMapper<ProcessGroupStatus>() {
@Override
public Long getValue(final ProcessGroupStatus status) {
return status.getQueuedCount().longValue();
}
})),
QUEUED_COUNT(new StandardMetricDescriptor<ProcessGroupStatus>("queuedCount",
"Queued Count",
"The number of FlowFiles queued in all Connections of this Process Group",
Formatter.COUNT,
s -> s.getQueuedCount().longValue())),
TASK_MILLIS(new StandardMetricDescriptor<ProcessGroupStatus>("taskMillis", "Total Task Duration (5 mins)",
TASK_MILLIS(new StandardMetricDescriptor<ProcessGroupStatus>("taskMillis",
"Total Task Duration (5 mins)",
"The total number of thread-milliseconds that the Processors within this ProcessGroup have used to complete their tasks in the past 5 minutes",
Formatter.DURATION, new ValueMapper<ProcessGroupStatus>() {
@Override
public Long getValue(final ProcessGroupStatus status) {
return calculateTaskMillis(status);
}
}));
Formatter.DURATION,
s -> calculateTaskMillis(s)));
private MetricDescriptor<ProcessGroupStatus> descriptor;

View File

@ -29,130 +29,78 @@ public enum ProcessorStatusDescriptor {
"Bytes Read (5 mins)",
"The total number of bytes read from the Content Repository by this Processor in the past 5 minutes",
Formatter.DATA_SIZE,
new ValueMapper<ProcessorStatus>() {
@Override
public Long getValue(final ProcessorStatus status) {
return status.getBytesRead();
}
})),
s -> s.getBytesRead())),
BYTES_WRITTEN(new StandardMetricDescriptor<ProcessorStatus>(
"bytesWritten",
"Bytes Written (5 mins)",
"The total number of bytes written to the Content Repository by this Processor in the past 5 minutes",
Formatter.DATA_SIZE,
new ValueMapper<ProcessorStatus>() {
@Override
public Long getValue(final ProcessorStatus status) {
return status.getBytesWritten();
}
})),
s -> s.getBytesWritten())),
BYTES_TRANSFERRED(new StandardMetricDescriptor<ProcessorStatus>(
"bytesTransferred",
"Bytes Transferred (5 mins)",
"The total number of bytes read from or written to the Content Repository by this Processor in the past 5 minutes",
Formatter.DATA_SIZE,
new ValueMapper<ProcessorStatus>() {
@Override
public Long getValue(final ProcessorStatus status) {
return status.getBytesRead() + status.getBytesWritten();
}
})),
s -> s.getBytesRead() + s.getBytesWritten())),
INPUT_BYTES(new StandardMetricDescriptor<ProcessorStatus>(
"inputBytes",
"Bytes In (5 mins)",
"The cumulative size of all FlowFiles that this Processor has pulled from its queues in the past 5 minutes",
Formatter.DATA_SIZE,
new ValueMapper<ProcessorStatus>() {
@Override
public Long getValue(final ProcessorStatus status) {
return status.getInputBytes();
}
})),
s -> s.getInputBytes())),
INPUT_COUNT(new StandardMetricDescriptor<ProcessorStatus>(
"inputCount",
"FlowFiles In (5 mins)",
"The number of FlowFiles that this Processor has pulled from its queues in the past 5 minutes",
Formatter.COUNT, new ValueMapper<ProcessorStatus>() {
@Override
public Long getValue(final ProcessorStatus status) {
return Long.valueOf(status.getInputCount());
}
})),
Formatter.COUNT,
s -> Long.valueOf(s.getInputCount()))),
OUTPUT_BYTES(new StandardMetricDescriptor<ProcessorStatus>(
"outputBytes",
"Bytes Out (5 mins)",
"The cumulative size of all FlowFiles that this Processor has transferred to downstream queues in the past 5 minutes",
Formatter.DATA_SIZE,
new ValueMapper<ProcessorStatus>() {
@Override
public Long getValue(final ProcessorStatus status) {
return status.getOutputBytes();
}
})),
s -> s.getOutputBytes())),
OUTPUT_COUNT(new StandardMetricDescriptor<ProcessorStatus>(
"outputCount",
"FlowFiles Out (5 mins)",
"The number of FlowFiles that this Processor has transferred to downstream queues in the past 5 minutes",
Formatter.COUNT,
new ValueMapper<ProcessorStatus>() {
@Override
public Long getValue(final ProcessorStatus status) {
return Long.valueOf(status.getOutputCount());
}
})),
s -> Long.valueOf(s.getOutputCount()))),
TASK_COUNT(new StandardMetricDescriptor<ProcessorStatus>(
"taskCount",
"Tasks (5 mins)",
"The number of tasks that this Processor has completed in the past 5 minutes",
Formatter.COUNT,
new ValueMapper<ProcessorStatus>() {
@Override
public Long getValue(final ProcessorStatus status) {
return Long.valueOf(status.getInvocations());
}
})),
s -> Long.valueOf(s.getInvocations()))),
TASK_MILLIS(new StandardMetricDescriptor<ProcessorStatus>(
"taskMillis",
"Total Task Duration (5 mins)",
"The total number of thread-milliseconds that the Processor has used to complete its tasks in the past 5 minutes",
Formatter.DURATION,
new ValueMapper<ProcessorStatus>() {
@Override
public Long getValue(final ProcessorStatus status) {
return TimeUnit.MILLISECONDS.convert(status.getProcessingNanos(), TimeUnit.NANOSECONDS);
}
})),
s -> TimeUnit.MILLISECONDS.convert(s.getProcessingNanos(), TimeUnit.NANOSECONDS))),
FLOWFILES_REMOVED(new StandardMetricDescriptor<ProcessorStatus>(
"flowFilesRemoved",
"FlowFiles Removed (5 mins)",
"The total number of FlowFiles removed by this Processor in the last 5 minutes",
Formatter.COUNT,
new ValueMapper<ProcessorStatus>() {
@Override
public Long getValue(final ProcessorStatus status) {
return Long.valueOf(status.getFlowFilesRemoved());
}
})),
s -> Long.valueOf(s.getFlowFilesRemoved()))),
AVERAGE_LINEAGE_DURATION(new StandardMetricDescriptor<ProcessorStatus>(
"averageLineageDuration",
"Average Lineage Duration (5 mins)",
"The average amount of time that a FlowFile took to process (from receipt until this Processor finished processing it) in the past 5 minutes.",
Formatter.DURATION,
new ValueMapper<ProcessorStatus>() {
@Override
public Long getValue(final ProcessorStatus status) {
return status.getAverageLineageDuration(TimeUnit.MILLISECONDS);
}
}, new ValueReducer<StatusSnapshot, Long>() {
s -> s.getAverageLineageDuration(TimeUnit.MILLISECONDS),
new ValueReducer<StatusSnapshot, Long>() {
@Override
public Long reduce(final List<StatusSnapshot> values) {
long millis = 0L;
@ -179,12 +127,7 @@ public enum ProcessorStatusDescriptor {
"Average Task Duration",
"The average duration it took this Processor to complete a task, as averaged over the past 5 minutes",
Formatter.DURATION,
new ValueMapper<ProcessorStatus>() {
@Override
public Long getValue(final ProcessorStatus status) {
return status.getInvocations() == 0 ? 0 : TimeUnit.MILLISECONDS.convert(status.getProcessingNanos(), TimeUnit.NANOSECONDS) / status.getInvocations();
}
},
s -> s.getInvocations() == 0 ? 0 : TimeUnit.MILLISECONDS.convert(s.getProcessingNanos(), TimeUnit.NANOSECONDS) / s.getInvocations(),
new ValueReducer<StatusSnapshot, Long>() {
@Override
public Long reduce(final List<StatusSnapshot> values) {

View File

@ -24,58 +24,47 @@ import org.apache.nifi.controller.status.RemoteProcessGroupStatus;
import org.apache.nifi.controller.status.history.MetricDescriptor.Formatter;
public enum RemoteProcessGroupStatusDescriptor {
SENT_BYTES(new StandardMetricDescriptor<RemoteProcessGroupStatus>("sentBytes", "Bytes Sent (5 mins)",
"The cumulative size of all FlowFiles that have been successfully sent to the remote system in the past 5 minutes", Formatter.DATA_SIZE, new ValueMapper<RemoteProcessGroupStatus>() {
@Override
public Long getValue(final RemoteProcessGroupStatus status) {
return status.getSentContentSize();
}
})),
SENT_BYTES(new StandardMetricDescriptor<RemoteProcessGroupStatus>("sentBytes",
"Bytes Sent (5 mins)",
"The cumulative size of all FlowFiles that have been successfully sent to the remote system in the past 5 minutes",
Formatter.DATA_SIZE,
s -> s.getSentContentSize())),
SENT_COUNT(new StandardMetricDescriptor<RemoteProcessGroupStatus>("sentCount", "FlowFiles Sent (5 mins)",
"The number of FlowFiles that have been successfully sent to the remote system in the past 5 minutes", Formatter.COUNT, new ValueMapper<RemoteProcessGroupStatus>() {
@Override
public Long getValue(final RemoteProcessGroupStatus status) {
return Long.valueOf(status.getSentCount().longValue());
}
})),
SENT_COUNT(new StandardMetricDescriptor<RemoteProcessGroupStatus>("sentCount",
"FlowFiles Sent (5 mins)",
"The number of FlowFiles that have been successfully sent to the remote system in the past 5 minutes",
Formatter.COUNT,
s -> s.getSentCount().longValue())),
RECEIVED_BYTES(new StandardMetricDescriptor<RemoteProcessGroupStatus>("receivedBytes", "Bytes Received (5 mins)",
"The cumulative size of all FlowFiles that have been received from the remote system in the past 5 minutes", Formatter.DATA_SIZE, new ValueMapper<RemoteProcessGroupStatus>() {
@Override
public Long getValue(final RemoteProcessGroupStatus status) {
return status.getReceivedContentSize();
}
})),
RECEIVED_BYTES(new StandardMetricDescriptor<RemoteProcessGroupStatus>("receivedBytes",
"Bytes Received (5 mins)",
"The cumulative size of all FlowFiles that have been received from the remote system in the past 5 minutes",
Formatter.DATA_SIZE,
s -> s.getReceivedContentSize())),
RECEIVED_COUNT(new StandardMetricDescriptor<RemoteProcessGroupStatus>("receivedCount", "FlowFiles Received (5 mins)",
"The number of FlowFiles that have been received from the remote system in the past 5 minutes", Formatter.COUNT, new ValueMapper<RemoteProcessGroupStatus>() {
@Override
public Long getValue(final RemoteProcessGroupStatus status) {
return Long.valueOf(status.getReceivedCount().longValue());
}
})),
RECEIVED_COUNT(new StandardMetricDescriptor<RemoteProcessGroupStatus>("receivedCount",
"FlowFiles Received (5 mins)",
"The number of FlowFiles that have been received from the remote system in the past 5 minutes",
Formatter.COUNT,
s -> s.getReceivedCount().longValue())),
RECEIVED_BYTES_PER_SECOND(new StandardMetricDescriptor<RemoteProcessGroupStatus>("receivedBytesPerSecond", "Received Bytes Per Second",
RECEIVED_BYTES_PER_SECOND(new StandardMetricDescriptor<RemoteProcessGroupStatus>("receivedBytesPerSecond",
"Received Bytes Per Second",
"The data rate at which data was received from the remote system in the past 5 minutes in terms of Bytes Per Second",
Formatter.DATA_SIZE, new ValueMapper<RemoteProcessGroupStatus>() {
@Override
public Long getValue(final RemoteProcessGroupStatus status) {
return Long.valueOf(status.getReceivedContentSize().longValue() / 300L);
}
})),
Formatter.DATA_SIZE,
s -> s.getReceivedContentSize().longValue() / 300L)),
SENT_BYTES_PER_SECOND(new StandardMetricDescriptor<RemoteProcessGroupStatus>("sentBytesPerSecond", "Sent Bytes Per Second",
"The data rate at which data was received from the remote system in the past 5 minutes in terms of Bytes Per Second", Formatter.DATA_SIZE, new ValueMapper<RemoteProcessGroupStatus>() {
@Override
public Long getValue(final RemoteProcessGroupStatus status) {
return Long.valueOf(status.getSentContentSize().longValue() / 300L);
}
})),
SENT_BYTES_PER_SECOND(new StandardMetricDescriptor<RemoteProcessGroupStatus>("sentBytesPerSecond",
"Sent Bytes Per Second",
"The data rate at which data was received from the remote system in the past 5 minutes in terms of Bytes Per Second",
Formatter.DATA_SIZE,
s -> s.getSentContentSize().longValue() / 300L)),
TOTAL_BYTES_PER_SECOND(new StandardMetricDescriptor<RemoteProcessGroupStatus>("totalBytesPerSecond", "Total Bytes Per Second",
TOTAL_BYTES_PER_SECOND(new StandardMetricDescriptor<RemoteProcessGroupStatus>("totalBytesPerSecond",
"Total Bytes Per Second",
"The sum of the send and receive data rate from the remote system in the past 5 minutes in terms of Bytes Per Second",
Formatter.DATA_SIZE, new ValueMapper<RemoteProcessGroupStatus>() {
Formatter.DATA_SIZE,
new ValueMapper<RemoteProcessGroupStatus>() {
@Override
public Long getValue(final RemoteProcessGroupStatus status) {
return Long.valueOf((status.getReceivedContentSize().longValue() + status.getSentContentSize().longValue()) / 300L);
@ -87,12 +76,8 @@ public enum RemoteProcessGroupStatusDescriptor {
"Average Lineage Duration (5 mins)",
"The average amount of time that a FlowFile took to process from receipt to drop in the past 5 minutes. For Processors that do not terminate FlowFiles, this value will be 0.",
Formatter.DURATION,
new ValueMapper<RemoteProcessGroupStatus>() {
@Override
public Long getValue(final RemoteProcessGroupStatus status) {
return status.getAverageLineageDuration(TimeUnit.MILLISECONDS);
}
}, new ValueReducer<StatusSnapshot, Long>() {
s -> s.getAverageLineageDuration(TimeUnit.MILLISECONDS),
new ValueReducer<StatusSnapshot, Long>() {
@Override
public Long reduce(final List<StatusSnapshot> values) {
long millis = 0L;

View File

@ -16,10 +16,7 @@
*/
package org.apache.nifi.controller.status.history;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Date;
import java.util.List;
import org.apache.nifi.controller.status.ConnectionStatus;
import org.apache.nifi.controller.status.ProcessGroupStatus;
@ -43,36 +40,6 @@ public class VolatileComponentStatusRepository implements ComponentStatusReposit
private volatile long lastCaptureTime = 0L;
private static final List<MetricDescriptor<ProcessorStatus>> PROCESSOR_METRIC_DESCRIPTORS;
private static final List<MetricDescriptor<ConnectionStatus>> CONNECTION_METRIC_DESCRIPTORS;
private static final List<MetricDescriptor<ProcessGroupStatus>> PROCESS_GROUP_METRIC_DESCRIPTORS;
private static final List<MetricDescriptor<RemoteProcessGroupStatus>> REMOTE_PROCESS_GROUP_METRIC_DESCRIPTORS;
static {
final List<MetricDescriptor<ProcessorStatus>> procFields = new ArrayList<>();
for (final ProcessorStatusDescriptor descriptor : ProcessorStatusDescriptor.values()) {
procFields.add(descriptor.getDescriptor());
}
PROCESSOR_METRIC_DESCRIPTORS = Collections.unmodifiableList(procFields);
final List<MetricDescriptor<ConnectionStatus>> connFields = new ArrayList<>();
for (final ConnectionStatusDescriptor descriptor : ConnectionStatusDescriptor.values()) {
connFields.add(descriptor.getDescriptor());
}
CONNECTION_METRIC_DESCRIPTORS = Collections.unmodifiableList(connFields);
final List<MetricDescriptor<ProcessGroupStatus>> groupFields = new ArrayList<>();
for (final ProcessGroupStatusDescriptor descriptor : ProcessGroupStatusDescriptor.values()) {
groupFields.add(descriptor.getDescriptor());
}
PROCESS_GROUP_METRIC_DESCRIPTORS = Collections.unmodifiableList(groupFields);
final List<MetricDescriptor<RemoteProcessGroupStatus>> remoteGroupFields = new ArrayList<>();
for (final RemoteProcessGroupStatusDescriptor descriptor : RemoteProcessGroupStatusDescriptor.values()) {
remoteGroupFields.add(descriptor.getDescriptor());
}
REMOTE_PROCESS_GROUP_METRIC_DESCRIPTORS = Collections.unmodifiableList(remoteGroupFields);
}
public VolatileComponentStatusRepository() {
final NiFiProperties properties = NiFiProperties.getInstance();
@ -249,26 +216,4 @@ public class VolatileComponentStatusRepository implements ComponentStatusReposit
return statusReport;
}
}
@Override
public List<MetricDescriptor<ConnectionStatus>> getConnectionMetricDescriptors() {
return CONNECTION_METRIC_DESCRIPTORS;
}
@Override
public List<MetricDescriptor<ProcessGroupStatus>> getProcessGroupMetricDescriptors() {
return PROCESS_GROUP_METRIC_DESCRIPTORS;
}
@Override
public List<MetricDescriptor<RemoteProcessGroupStatus>> getRemoteProcessGroupMetricDescriptors() {
return REMOTE_PROCESS_GROUP_METRIC_DESCRIPTORS;
}
@Override
public List<MetricDescriptor<ProcessorStatus>> getProcessorMetricDescriptors() {
return PROCESSOR_METRIC_DESCRIPTORS;
}
}

View File

@ -53,10 +53,12 @@ import org.apache.nifi.remote.RemoteGroupPort;
import org.apache.nifi.remote.RootGroupPort;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.util.ReflectionUtils;
import org.apache.nifi.web.Revision;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashSet;
@ -1765,7 +1767,7 @@ public final class StandardProcessGroup implements ProcessGroup {
verifyContents(snippet);
final Set<Connectable> connectables = getAllConnectables(snippet);
final Set<String> connectionIdsToRemove = new HashSet<>(replaceNullWithEmptySet(snippet.getConnections()));
final Set<String> connectionIdsToRemove = new HashSet<>(getKeys(snippet.getConnections()));
// Remove all connections that are the output of any Connectable.
for (final Connectable connectable : connectables) {
for (final Connection conn : connectable.getConnections()) {
@ -1782,7 +1784,7 @@ public final class StandardProcessGroup implements ProcessGroup {
}
// verify that all processors are stopped and have no active threads
for (final String procId : snippet.getProcessors()) {
for (final String procId : snippet.getProcessors().keySet()) {
final ProcessorNode procNode = getProcessor(procId);
if (procNode.isRunning()) {
throw new IllegalStateException(procNode + " cannot be removed because it is running");
@ -1794,7 +1796,7 @@ public final class StandardProcessGroup implements ProcessGroup {
}
// verify that none of the connectables have incoming connections that are not in the Snippet.
final Set<String> connectionIds = snippet.getConnections();
final Set<String> connectionIds = snippet.getConnections().keySet();
for (final Connectable connectable : connectables) {
for (final Connection conn : connectable.getIncomingConnections()) {
if (!connectionIds.contains(conn.getIdentifier()) && !connectables.contains(conn.getSource())) {
@ -1805,7 +1807,7 @@ public final class StandardProcessGroup implements ProcessGroup {
}
// verify that all of the ProcessGroups in the snippet are empty
for (final String groupId : snippet.getProcessGroups()) {
for (final String groupId : snippet.getProcessGroups().keySet()) {
final ProcessGroup toRemove = getProcessGroup(groupId);
toRemove.verifyCanDelete(true);
}
@ -1813,25 +1815,25 @@ public final class StandardProcessGroup implements ProcessGroup {
for (final String id : connectionIdsToRemove) {
removeConnection(connections.get(id));
}
for (final String id : replaceNullWithEmptySet(snippet.getInputPorts())) {
for (final String id : getKeys(snippet.getInputPorts())) {
removeInputPort(inputPorts.get(id));
}
for (final String id : replaceNullWithEmptySet(snippet.getOutputPorts())) {
for (final String id : getKeys(snippet.getOutputPorts())) {
removeOutputPort(outputPorts.get(id));
}
for (final String id : replaceNullWithEmptySet(snippet.getFunnels())) {
for (final String id : getKeys(snippet.getFunnels())) {
removeFunnel(funnels.get(id));
}
for (final String id : replaceNullWithEmptySet(snippet.getLabels())) {
for (final String id : getKeys(snippet.getLabels())) {
removeLabel(labels.get(id));
}
for (final String id : replaceNullWithEmptySet(snippet.getProcessors())) {
for (final String id : getKeys(snippet.getProcessors())) {
removeProcessor(processors.get(id));
}
for (final String id : replaceNullWithEmptySet(snippet.getRemoteProcessGroups())) {
for (final String id : getKeys(snippet.getRemoteProcessGroups())) {
removeRemoteProcessGroup(remoteGroups.get(id));
}
for (final String id : replaceNullWithEmptySet(snippet.getProcessGroups())) {
for (final String id : getKeys(snippet.getProcessGroups())) {
removeProcessGroup(processGroups.get(id));
}
} finally {
@ -1839,10 +1841,11 @@ public final class StandardProcessGroup implements ProcessGroup {
}
}
private Set<String> replaceNullWithEmptySet(final Set<String> toReplace) {
return (toReplace == null) ? new HashSet<String>() : toReplace;
private Set<String> getKeys(final Map<String, Revision> map) {
return (map == null) ? Collections.emptySet() : map.keySet();
}
@Override
public void move(final Snippet snippet, final ProcessGroup destination) {
writeLock.lock();
@ -1861,28 +1864,28 @@ public final class StandardProcessGroup implements ProcessGroup {
throw new IllegalStateException("Cannot move Ports into the root group");
}
for (final String id : replaceNullWithEmptySet(snippet.getInputPorts())) {
for (final String id : getKeys(snippet.getInputPorts())) {
destination.addInputPort(inputPorts.remove(id));
}
for (final String id : replaceNullWithEmptySet(snippet.getOutputPorts())) {
for (final String id : getKeys(snippet.getOutputPorts())) {
destination.addOutputPort(outputPorts.remove(id));
}
for (final String id : replaceNullWithEmptySet(snippet.getFunnels())) {
for (final String id : getKeys(snippet.getFunnels())) {
destination.addFunnel(funnels.remove(id));
}
for (final String id : replaceNullWithEmptySet(snippet.getLabels())) {
for (final String id : getKeys(snippet.getLabels())) {
destination.addLabel(labels.remove(id));
}
for (final String id : replaceNullWithEmptySet(snippet.getProcessGroups())) {
for (final String id : getKeys(snippet.getProcessGroups())) {
destination.addProcessGroup(processGroups.remove(id));
}
for (final String id : replaceNullWithEmptySet(snippet.getProcessors())) {
for (final String id : getKeys(snippet.getProcessors())) {
destination.addProcessor(processors.remove(id));
}
for (final String id : replaceNullWithEmptySet(snippet.getRemoteProcessGroups())) {
for (final String id : getKeys(snippet.getRemoteProcessGroups())) {
destination.addRemoteProcessGroup(remoteGroups.remove(id));
}
for (final String id : replaceNullWithEmptySet(snippet.getConnections())) {
for (final String id : getKeys(snippet.getConnections())) {
destination.inheritConnection(connections.remove(id));
}
} finally {
@ -1892,16 +1895,16 @@ public final class StandardProcessGroup implements ProcessGroup {
private Set<Connectable> getAllConnectables(final Snippet snippet) {
final Set<Connectable> connectables = new HashSet<>();
for (final String id : replaceNullWithEmptySet(snippet.getInputPorts())) {
for (final String id : getKeys(snippet.getInputPorts())) {
connectables.add(getInputPort(id));
}
for (final String id : replaceNullWithEmptySet(snippet.getOutputPorts())) {
for (final String id : getKeys(snippet.getOutputPorts())) {
connectables.add(getOutputPort(id));
}
for (final String id : replaceNullWithEmptySet(snippet.getFunnels())) {
for (final String id : getKeys(snippet.getFunnels())) {
connectables.add(getFunnel(id));
}
for (final String id : replaceNullWithEmptySet(snippet.getProcessors())) {
for (final String id : getKeys(snippet.getProcessors())) {
connectables.add(getProcessor(id));
}
return connectables;
@ -1910,13 +1913,13 @@ public final class StandardProcessGroup implements ProcessGroup {
private boolean isDisconnected(final Snippet snippet) {
final Set<Connectable> connectables = getAllConnectables(snippet);
for (final String id : replaceNullWithEmptySet(snippet.getRemoteProcessGroups())) {
for (final String id : getKeys(snippet.getRemoteProcessGroups())) {
final RemoteProcessGroup remoteGroup = getRemoteProcessGroup(id);
connectables.addAll(remoteGroup.getInputPorts());
connectables.addAll(remoteGroup.getOutputPorts());
}
final Set<String> connectionIds = snippet.getConnections();
final Set<String> connectionIds = snippet.getConnections().keySet();
for (final Connectable connectable : connectables) {
for (final Connection conn : connectable.getIncomingConnections()) {
if (!connectionIds.contains(conn.getIdentifier())) {
@ -1932,7 +1935,7 @@ public final class StandardProcessGroup implements ProcessGroup {
}
final Set<Connectable> recursiveConnectables = new HashSet<>(connectables);
for (final String id : snippet.getProcessGroups()) {
for (final String id : snippet.getProcessGroups().keySet()) {
final ProcessGroup childGroup = getProcessGroup(id);
recursiveConnectables.addAll(findAllConnectables(childGroup, true));
}
@ -1977,14 +1980,14 @@ public final class StandardProcessGroup implements ProcessGroup {
private void verifyContents(final Snippet snippet) throws NullPointerException, IllegalStateException {
requireNonNull(snippet);
verifyAllKeysExist(snippet.getInputPorts(), inputPorts, "Input Port");
verifyAllKeysExist(snippet.getOutputPorts(), outputPorts, "Output Port");
verifyAllKeysExist(snippet.getFunnels(), funnels, "Funnel");
verifyAllKeysExist(snippet.getLabels(), labels, "Label");
verifyAllKeysExist(snippet.getProcessGroups(), processGroups, "Process Group");
verifyAllKeysExist(snippet.getProcessors(), processors, "Processor");
verifyAllKeysExist(snippet.getRemoteProcessGroups(), remoteGroups, "Remote Process Group");
verifyAllKeysExist(snippet.getConnections(), connections, "Connection");
verifyAllKeysExist(snippet.getInputPorts().keySet(), inputPorts, "Input Port");
verifyAllKeysExist(snippet.getOutputPorts().keySet(), outputPorts, "Output Port");
verifyAllKeysExist(snippet.getFunnels().keySet(), funnels, "Funnel");
verifyAllKeysExist(snippet.getLabels().keySet(), labels, "Label");
verifyAllKeysExist(snippet.getProcessGroups().keySet(), processGroups, "Process Group");
verifyAllKeysExist(snippet.getProcessors().keySet(), processors, "Processor");
verifyAllKeysExist(snippet.getRemoteProcessGroups().keySet(), remoteGroups, "Remote Process Group");
verifyAllKeysExist(snippet.getConnections().keySet(), connections, "Connection");
}
/**
@ -2104,7 +2107,7 @@ public final class StandardProcessGroup implements ProcessGroup {
throw new IllegalStateException("One or more components within the snippet is connected to a component outside of the snippet. Only a disconnected snippet may be moved.");
}
for (final String id : snippet.getConnections()) {
for (final String id : snippet.getConnections().keySet()) {
final Connection connection = getConnection(id);
if (connection == null) {
throw new IllegalStateException("Snippet references Connection with ID " + id + ", which does not exist in this ProcessGroup");
@ -2113,7 +2116,7 @@ public final class StandardProcessGroup implements ProcessGroup {
connection.verifyCanDelete();
}
for (final String id : snippet.getFunnels()) {
for (final String id : snippet.getFunnels().keySet()) {
final Funnel funnel = getFunnel(id);
if (funnel == null) {
throw new IllegalStateException("Snippet references Funnel with ID " + id + ", which does not exist in this ProcessGroup");
@ -2122,7 +2125,7 @@ public final class StandardProcessGroup implements ProcessGroup {
funnel.verifyCanDelete(true);
}
for (final String id : snippet.getInputPorts()) {
for (final String id : snippet.getInputPorts().keySet()) {
final Port port = getInputPort(id);
if (port == null) {
throw new IllegalStateException("Snippet references Input Port with ID " + id + ", which does not exist in this ProcessGroup");
@ -2131,14 +2134,14 @@ public final class StandardProcessGroup implements ProcessGroup {
port.verifyCanDelete(true);
}
for (final String id : snippet.getLabels()) {
for (final String id : snippet.getLabels().keySet()) {
final Label label = getLabel(id);
if (label == null) {
throw new IllegalStateException("Snippet references Label with ID " + id + ", which does not exist in this ProcessGroup");
}
}
for (final String id : snippet.getOutputPorts()) {
for (final String id : snippet.getOutputPorts().keySet()) {
final Port port = getOutputPort(id);
if (port == null) {
throw new IllegalStateException("Snippet references Output Port with ID " + id + ", which does not exist in this ProcessGroup");
@ -2146,7 +2149,7 @@ public final class StandardProcessGroup implements ProcessGroup {
port.verifyCanDelete(true);
}
for (final String id : snippet.getProcessGroups()) {
for (final String id : snippet.getProcessGroups().keySet()) {
final ProcessGroup group = getProcessGroup(id);
if (group == null) {
throw new IllegalStateException("Snippet references Process Group with ID " + id + ", which does not exist in this ProcessGroup");
@ -2154,7 +2157,7 @@ public final class StandardProcessGroup implements ProcessGroup {
group.verifyCanDelete(true);
}
for (final String id : snippet.getProcessors()) {
for (final String id : snippet.getProcessors().keySet()) {
final ProcessorNode processor = getProcessor(id);
if (processor == null) {
throw new IllegalStateException("Snippet references Processor with ID " + id + ", which does not exist in this ProcessGroup");
@ -2162,7 +2165,7 @@ public final class StandardProcessGroup implements ProcessGroup {
processor.verifyCanDelete(true);
}
for (final String id : snippet.getRemoteProcessGroups()) {
for (final String id : snippet.getRemoteProcessGroups().keySet()) {
final RemoteProcessGroup group = getRemoteProcessGroup(id);
if (group == null) {
throw new IllegalStateException("Snippet references Remote Process Group with ID " + id + ", which does not exist in this ProcessGroup");
@ -2192,7 +2195,7 @@ public final class StandardProcessGroup implements ProcessGroup {
throw new IllegalStateException("Cannot move Ports from the Root Group to a Non-Root Group");
}
for (final String id : snippet.getInputPorts()) {
for (final String id : snippet.getInputPorts().keySet()) {
final Port port = getInputPort(id);
final String portName = port.getName();
@ -2201,7 +2204,7 @@ public final class StandardProcessGroup implements ProcessGroup {
}
}
for (final String id : snippet.getOutputPorts()) {
for (final String id : snippet.getOutputPorts().keySet()) {
final Port port = getOutputPort(id);
final String portName = port.getName();

View File

@ -16,6 +16,7 @@
*/
package org.apache.nifi.persistence;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
@ -56,6 +57,15 @@ public interface FlowConfigurationDAO {
*/
void load(OutputStream os) throws IOException;
/**
* Loads the stored flow into the given stream, optionally compressed
*
* @param os the Output Stream to write the flow to
* @param compressed whether or not the data should be gzipped
* @throws IOException if unable to load the flow
*/
void load(OutputStream os, boolean compressed) throws IOException;
/**
* Saves the given stream as the stored flow.
*
@ -96,4 +106,11 @@ public interface FlowConfigurationDAO {
*/
void save(FlowController flow, boolean archive) throws IOException;
/**
* Creates a File that can be used to write an archive to. The file will not actually exist on disk.
*
* @return a File that can be used to write an archive to
* @throws IOException if unable to access the required directories
*/
File createArchiveFile() throws IOException;
}

View File

@ -84,6 +84,15 @@ public final class StandardXMLFlowConfigurationDAO implements FlowConfigurationD
}
}
@Override
public void load(final OutputStream os, final boolean compressed) throws IOException {
if (compressed) {
Files.copy(flowXmlPath, os);
} else {
load(os);
}
}
@Override
public synchronized void save(final InputStream is) throws IOException {
try (final OutputStream outStream = Files.newOutputStream(flowXmlPath, StandardOpenOption.WRITE, StandardOpenOption.CREATE);
@ -140,15 +149,8 @@ public final class StandardXMLFlowConfigurationDAO implements FlowConfigurationD
if (archive) {
try {
final String archiveDirVal = NiFiProperties.getInstance().getProperty(CONFIGURATION_ARCHIVE_DIR_KEY);
final Path archiveDir = (archiveDirVal == null || archiveDirVal.equals("")) ? configFile.getParent().resolve("archive") : new File(archiveDirVal).toPath();
Files.createDirectories(archiveDir);
if (!Files.isDirectory(archiveDir)) {
throw new IOException("Archive directory doesn't appear to be a directory " + archiveDir);
}
final Path archiveFile = archiveDir.resolve(System.nanoTime() + "-" + configFile.toFile().getName());
Files.copy(configFile, archiveFile);
final File archiveFile = createArchiveFile();
Files.copy(configFile, archiveFile.toPath());
} catch (final Exception ex) {
LOG.warn("Unable to archive flow configuration as requested due to " + ex);
if (LOG.isDebugEnabled()) {
@ -158,4 +160,16 @@ public final class StandardXMLFlowConfigurationDAO implements FlowConfigurationD
}
}
@Override
public File createArchiveFile() throws IOException {
final String archiveDirVal = NiFiProperties.getInstance().getProperty(CONFIGURATION_ARCHIVE_DIR_KEY);
final Path archiveDir = (archiveDirVal == null || archiveDirVal.equals("")) ? flowXmlPath.getParent().resolve("archive") : new File(archiveDirVal).toPath();
Files.createDirectories(archiveDir);
if (!Files.isDirectory(archiveDir)) {
throw new IOException("Archive directory doesn't appear to be a directory " + archiveDir);
}
final Path archiveFile = archiveDir.resolve(System.nanoTime() + "-" + flowXmlPath.toFile().getName());
return archiveFile.toFile();
}
}

View File

@ -115,4 +115,10 @@ public interface FlowService extends LifeCycle {
*/
FlowController getController();
/**
* Creates a copy of the current flow and saves it in the configured 'archive' directory
*
* @throws IOException if unable to write to the archive directory
*/
void archiveFlow() throws IOException;
}

View File

@ -169,7 +169,6 @@ nifi.zookeeper.connect.string=${nifi.zookeeper.connect.string}
nifi.zookeeper.connect.timeout=${nifi.zookeeper.connect.timeout}
nifi.zookeeper.session.timeout=${nifi.zookeeper.session.timeout}
nifi.zookeeper.root.node=${nifi.zookeeper.root.node}
nifi.zookeeper.access.control=${nifi.zookeeper.access.control}
# cluster manager properties (only configure for cluster manager) #
nifi.cluster.is.manager=${nifi.cluster.is.manager}

View File

@ -271,7 +271,7 @@ public class SnippetAuditor extends NiFiAuditor {
// create move audit records for all items in this snippet
final Collection<Action> actions = new ArrayList<>();
for (String id : snippet.getProcessors()) {
for (String id : snippet.getProcessors().keySet()) {
final ProcessorNode processor = processorDAO.getProcessor(id);
final Action action = processorAuditor.generateAuditRecord(processor, Operation.Move, createMoveDetails(previousGroupId, groupId, logger));
if (action != null) {
@ -279,7 +279,7 @@ public class SnippetAuditor extends NiFiAuditor {
}
}
for (String id : snippet.getFunnels()) {
for (String id : snippet.getFunnels().keySet()) {
final Funnel funnel = funnelDAO.getFunnel(id);
final Action action = funnelAuditor.generateAuditRecord(funnel, Operation.Move, createMoveDetails(previousGroupId, groupId, logger));
if (action != null) {
@ -287,7 +287,7 @@ public class SnippetAuditor extends NiFiAuditor {
}
}
for (String id : snippet.getInputPorts()) {
for (String id : snippet.getInputPorts().keySet()) {
final Port port = inputPortDAO.getPort(id);
final Action action = portAuditor.generateAuditRecord(port, Operation.Move, createMoveDetails(previousGroupId, groupId, logger));
if (action != null) {
@ -295,7 +295,7 @@ public class SnippetAuditor extends NiFiAuditor {
}
}
for (String id : snippet.getOutputPorts()) {
for (String id : snippet.getOutputPorts().keySet()) {
final Port port = outputPortDAO.getPort(id);
final Action action = portAuditor.generateAuditRecord(port, Operation.Move, createMoveDetails(previousGroupId, groupId, logger));
if (action != null) {
@ -303,7 +303,7 @@ public class SnippetAuditor extends NiFiAuditor {
}
}
for (String id : snippet.getRemoteProcessGroups()) {
for (String id : snippet.getRemoteProcessGroups().keySet()) {
final RemoteProcessGroup remoteProcessGroup = remoteProcessGroupDAO.getRemoteProcessGroup(id);
final Action action = remoteProcessGroupAuditor.generateAuditRecord(remoteProcessGroup, Operation.Move, createMoveDetails(previousGroupId, groupId, logger));
if (action != null) {
@ -311,7 +311,7 @@ public class SnippetAuditor extends NiFiAuditor {
}
}
for (String id : snippet.getProcessGroups()) {
for (String id : snippet.getProcessGroups().keySet()) {
final ProcessGroupDAO processGroupDAO = getProcessGroupDAO();
final ProcessGroup processGroup = processGroupDAO.getProcessGroup(id);
final Action action = processGroupAuditor.generateAuditRecord(processGroup, Operation.Move, createMoveDetails(previousGroupId, groupId, logger));
@ -320,7 +320,7 @@ public class SnippetAuditor extends NiFiAuditor {
}
}
for (String id : snippet.getConnections()) {
for (String id : snippet.getConnections().keySet()) {
final Connection connection = connectionDAO.getConnection(id);
final Action action = relationshipAuditor.generateAuditRecordForConnection(connection, Operation.Move, createMoveDetails(previousGroupId, groupId, logger));
if (action != null) {
@ -356,38 +356,38 @@ public class SnippetAuditor extends NiFiAuditor {
if (snippet.isLinked()) {
// locate all the components being removed
final Set<Funnel> funnels = new HashSet<>();
for (String id : snippet.getFunnels()) {
for (String id : snippet.getFunnels().keySet()) {
funnels.add(funnelDAO.getFunnel(id));
}
final Set<Port> inputPorts = new HashSet<>();
for (String id : snippet.getInputPorts()) {
for (String id : snippet.getInputPorts().keySet()) {
inputPorts.add(inputPortDAO.getPort(id));
}
final Set<Port> outputPorts = new HashSet<>();
for (String id : snippet.getOutputPorts()) {
for (String id : snippet.getOutputPorts().keySet()) {
outputPorts.add(outputPortDAO.getPort(id));
}
final Set<RemoteProcessGroup> remoteProcessGroups = new HashSet<>();
for (String id : snippet.getRemoteProcessGroups()) {
for (String id : snippet.getRemoteProcessGroups().keySet()) {
remoteProcessGroups.add(remoteProcessGroupDAO.getRemoteProcessGroup(id));
}
final Set<ProcessGroup> processGroups = new HashSet<>();
final ProcessGroupDAO processGroupDAO = getProcessGroupDAO();
for (String id : snippet.getProcessGroups()) {
for (String id : snippet.getProcessGroups().keySet()) {
processGroups.add(processGroupDAO.getProcessGroup(id));
}
final Set<ProcessorNode> processors = new HashSet<>();
for (String id : snippet.getProcessors()) {
for (String id : snippet.getProcessors().keySet()) {
processors.add(processorDAO.getProcessor(id));
}
final Set<Connection> connections = new HashSet<>();
for (String id : snippet.getConnections()) {
for (String id : snippet.getConnections().keySet()) {
connections.add(connectionDAO.getConnection(id));
}

View File

@ -16,6 +16,10 @@
*/
package org.apache.nifi.web;
import java.util.Date;
import java.util.List;
import java.util.Set;
import org.apache.nifi.controller.ScheduledState;
import org.apache.nifi.controller.repository.claim.ContentDirection;
import org.apache.nifi.controller.service.ControllerServiceState;
@ -28,7 +32,6 @@ import org.apache.nifi.web.api.dto.ConnectionDTO;
import org.apache.nifi.web.api.dto.ControllerConfigurationDTO;
import org.apache.nifi.web.api.dto.ControllerDTO;
import org.apache.nifi.web.api.dto.ControllerServiceDTO;
import org.apache.nifi.web.api.dto.ControllerServiceReferencingComponentDTO;
import org.apache.nifi.web.api.dto.CounterDTO;
import org.apache.nifi.web.api.dto.CountersDTO;
import org.apache.nifi.web.api.dto.DocumentedTypeDTO;
@ -53,7 +56,6 @@ import org.apache.nifi.web.api.dto.TemplateDTO;
import org.apache.nifi.web.api.dto.action.ActionDTO;
import org.apache.nifi.web.api.dto.action.HistoryDTO;
import org.apache.nifi.web.api.dto.action.HistoryQueryDTO;
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.provenance.ProvenanceDTO;
import org.apache.nifi.web.api.dto.provenance.ProvenanceEventDTO;
@ -68,22 +70,29 @@ import org.apache.nifi.web.api.dto.status.ProcessorStatusDTO;
import org.apache.nifi.web.api.dto.status.RemoteProcessGroupStatusDTO;
import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
import org.apache.nifi.web.api.entity.ConnectionEntity;
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentsEntity;
import org.apache.nifi.web.api.entity.FlowEntity;
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.ProcessorEntity;
import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
import java.util.Date;
import java.util.List;
import java.util.Set;
import org.apache.nifi.web.api.entity.RemoteProcessGroupPortEntity;
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
import org.apache.nifi.web.api.entity.SnippetEntity;
/**
* Defines the NiFiServiceFacade interface.
*/
public interface NiFiServiceFacade {
// ----------------------------------------
// Synchronization methods
// ----------------------------------------
void claimRevision(Revision revision) throws InvalidRevisionException;
// ----------------------------------------
// Controller methods
// ----------------------------------------
@ -208,7 +217,7 @@ public interface NiFiServiceFacade {
* @param revision Revision to compare with current base revision
* @return snapshot
*/
ConfigurationSnapshot<Void> createArchive(Revision revision);
ProcessGroupEntity createArchive(Revision revision);
/**
* Sets the annotation data for a processor.
@ -219,7 +228,7 @@ public interface NiFiServiceFacade {
*
* @return snapshot
*/
ConfigurationSnapshot<ProcessorDTO> setProcessorAnnotationData(Revision revision, String processorId, String annotationData);
ProcessorEntity setProcessorAnnotationData(Revision revision, String processorId, String annotationData);
/**
* Returns the process group status.
@ -326,7 +335,7 @@ public interface NiFiServiceFacade {
* @param originY y
* @return snapshot
*/
ConfigurationSnapshot<FlowDTO> createTemplateInstance(Revision revision, String groupId, Double originX, Double originY, String templateId);
FlowEntity createTemplateInstance(Revision revision, String groupId, Double originX, Double originY, String templateId);
/**
* Gets the template with the specified id.
@ -535,6 +544,9 @@ public interface NiFiServiceFacade {
* @param connectionId The ID of the connection
* @return snapshot
*/
// TODO: Remove the id's from all of the delete methods. The Revision now contains
// the component ID. We can't do this yet though because we have to wait for the
// Revisions to be properly formed on the client/UI side
ConnectionEntity deleteConnection(Revision revision, String connectionId);
/**
@ -896,7 +908,7 @@ public interface NiFiServiceFacade {
* @param remoteProcessGroupPortDTO The RemoteProcessGroupPortDTO
* @return snapshot
*/
ConfigurationSnapshot<RemoteProcessGroupPortDTO> updateRemoteProcessGroupInputPort(Revision revision, String remoteProcessGroupId, RemoteProcessGroupPortDTO remoteProcessGroupPortDTO);
RemoteProcessGroupPortEntity updateRemoteProcessGroupInputPort(Revision revision, String remoteProcessGroupId, RemoteProcessGroupPortDTO remoteProcessGroupPortDTO);
/**
* Updates the specified remote process groups output port.
@ -906,7 +918,7 @@ public interface NiFiServiceFacade {
* @param remoteProcessGroupPortDTO The RemoteProcessGroupPortDTO
* @return snapshot
*/
ConfigurationSnapshot<RemoteProcessGroupPortDTO> updateRemoteProcessGroupOutputPort(Revision revision, String remoteProcessGroupId, RemoteProcessGroupPortDTO remoteProcessGroupPortDTO);
RemoteProcessGroupPortEntity updateRemoteProcessGroupOutputPort(Revision revision, String remoteProcessGroupId, RemoteProcessGroupPortDTO remoteProcessGroupPortDTO);
/**
* Verifies the remote process group can be deleted.
@ -1111,7 +1123,7 @@ public interface NiFiServiceFacade {
* @param controllerServiceDTO The controller service DTO
* @return The controller service DTO
*/
ConfigurationSnapshot<ControllerServiceDTO> createControllerService(Revision revision, ControllerServiceDTO controllerServiceDTO);
ControllerServiceEntity createControllerService(Revision revision, ControllerServiceDTO controllerServiceDTO);
/**
* Gets all controller services.
@ -1143,7 +1155,7 @@ public interface NiFiServiceFacade {
* @param controllerServiceId id
* @return service reference
*/
Set<ControllerServiceReferencingComponentDTO> getControllerServiceReferencingComponents(String controllerServiceId);
ControllerServiceReferencingComponentsEntity getControllerServiceReferencingComponents(String controllerServiceId);
/**
* Updates the referencing components for the specified controller service.
@ -1154,7 +1166,7 @@ public interface NiFiServiceFacade {
* @param controllerServiceState the value of state
* @return The referencing component dtos
*/
ConfigurationSnapshot<Set<ControllerServiceReferencingComponentDTO>> updateControllerServiceReferencingComponents(
ControllerServiceReferencingComponentsEntity updateControllerServiceReferencingComponents(
Revision revision, String controllerServiceId, ScheduledState scheduledState, ControllerServiceState controllerServiceState);
/**
@ -1164,7 +1176,7 @@ public interface NiFiServiceFacade {
* @param controllerServiceDTO The controller service DTO
* @return The controller service DTO
*/
ConfigurationSnapshot<ControllerServiceDTO> updateControllerService(Revision revision, ControllerServiceDTO controllerServiceDTO);
UpdateResult<ControllerServiceEntity> updateControllerService(Revision revision, ControllerServiceDTO controllerServiceDTO);
/**
* Deletes the specified label.
@ -1173,7 +1185,7 @@ public interface NiFiServiceFacade {
* @param controllerServiceId The controller service id
* @return snapshot
*/
ConfigurationSnapshot<Void> deleteControllerService(Revision revision, String controllerServiceId);
ControllerServiceEntity deleteControllerService(Revision revision, String controllerServiceId);
/**
* Verifies the specified controller service can be updated.
@ -1208,7 +1220,7 @@ public interface NiFiServiceFacade {
* @param reportingTaskDTO The reporting task DTO
* @return The reporting task DTO
*/
ConfigurationSnapshot<ReportingTaskDTO> createReportingTask(Revision revision, ReportingTaskDTO reportingTaskDTO);
ReportingTaskEntity createReportingTask(Revision revision, ReportingTaskDTO reportingTaskDTO);
/**
* Gets all reporting tasks.
@ -1241,7 +1253,7 @@ public interface NiFiServiceFacade {
* @param reportingTaskDTO The reporting task DTO
* @return The reporting task DTO
*/
ConfigurationSnapshot<ReportingTaskDTO> updateReportingTask(Revision revision, ReportingTaskDTO reportingTaskDTO);
UpdateResult<ReportingTaskEntity> updateReportingTask(Revision revision, ReportingTaskDTO reportingTaskDTO);
/**
* Deletes the specified reporting task.
@ -1250,7 +1262,7 @@ public interface NiFiServiceFacade {
* @param reportingTaskId The reporting task id
* @return snapshot
*/
ConfigurationSnapshot<Void> deleteReportingTask(Revision revision, String reportingTaskId);
ReportingTaskEntity deleteReportingTask(Revision revision, String reportingTaskId);
/**
* Verifies the specified reporting task can be updated.
@ -1313,7 +1325,7 @@ public interface NiFiServiceFacade {
* @param originY y
* @return snapshot
*/
ConfigurationSnapshot<FlowDTO> copySnippet(Revision revision, String groupId, String snippetId, Double originX, Double originY);
FlowEntity copySnippet(Revision revision, String groupId, String snippetId, Double originX, Double originY);
/**
* Creates a new snippet.
@ -1362,7 +1374,7 @@ public interface NiFiServiceFacade {
* @param snippetId snippet
* @return snapshot
*/
ConfigurationSnapshot<Void> deleteSnippet(Revision revision, String snippetId);
SnippetEntity deleteSnippet(Revision revision, String snippetId);
// ----------------------------------------
// Cluster methods

View File

@ -370,8 +370,8 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
}
processor = entity.getComponent();
} else {
final ConfigurationSnapshot<ProcessorDTO> response = serviceFacade.setProcessorAnnotationData(revision, id, annotationData);
processor = response.getConfiguration();
final ProcessorEntity entity = serviceFacade.setProcessorAnnotationData(revision, id, annotationData);
processor = entity.getComponent();
}
// return the processor info
@ -453,8 +453,8 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
controllerServiceDto.setId(id);
controllerServiceDto.setAnnotationData(annotationData);
final ConfigurationSnapshot<ControllerServiceDTO> response = serviceFacade.updateControllerService(revision, controllerServiceDto);
controllerService = response.getConfiguration();
final UpdateResult<ControllerServiceEntity> updateResult = serviceFacade.updateControllerService(revision, controllerServiceDto);
controllerService = updateResult.getResult().getControllerService();
} else {
// if this is a standalone instance the service should have been found above... there should
// no cluster to replicate the request to
@ -588,8 +588,8 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
reportingTaskDto.setId(id);
reportingTaskDto.setAnnotationData(annotationData);
final ConfigurationSnapshot<ReportingTaskDTO> response = serviceFacade.updateReportingTask(revision, reportingTaskDto);
reportingTask = response.getConfiguration();
final UpdateResult<ReportingTaskEntity> updateResult = serviceFacade.updateReportingTask(revision, reportingTaskDto);
reportingTask = updateResult.getResult().getReportingTask();
} else {
// if this is a standalone instance the task should have been found above... there should
// no cluster to replicate the request to

View File

@ -16,29 +16,14 @@
*/
package org.apache.nifi.web.api;
import com.sun.jersey.api.core.HttpContext;
import com.sun.jersey.api.representation.Form;
import com.sun.jersey.core.util.MultivaluedMapImpl;
import com.sun.jersey.server.impl.model.method.dispatch.FormDispatchProvider;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.lang3.builder.ReflectionToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle;
import org.apache.nifi.action.Action;
import org.apache.nifi.action.FlowChangeAction;
import org.apache.nifi.action.Operation;
import org.apache.nifi.authorization.user.NiFiUserDetails;
import org.apache.nifi.cluster.context.ClusterContext;
import org.apache.nifi.cluster.context.ClusterContextThreadLocal;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.api.entity.Entity;
import org.apache.nifi.web.api.request.ClientIdParameter;
import org.apache.nifi.web.security.jwt.JwtAuthenticationFilter;
import org.apache.nifi.web.util.WebUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.security.core.Authentication;
import org.springframework.security.core.context.SecurityContextHolder;
import java.io.Serializable;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.Collection;
import java.util.Enumeration;
import java.util.HashMap;
import java.util.Map;
import java.util.TreeMap;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
@ -50,15 +35,35 @@ import javax.ws.rs.core.Response.ResponseBuilder;
import javax.ws.rs.core.UriBuilder;
import javax.ws.rs.core.UriBuilderException;
import javax.ws.rs.core.UriInfo;
import java.io.Serializable;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.Collection;
import java.util.Enumeration;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.lang3.builder.ReflectionToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle;
import org.apache.nifi.action.Action;
import org.apache.nifi.action.FlowChangeAction;
import org.apache.nifi.action.Operation;
import org.apache.nifi.authorization.user.NiFiUserDetails;
import org.apache.nifi.cluster.context.ClusterContext;
import org.apache.nifi.cluster.context.ClusterContextThreadLocal;
import org.apache.nifi.cluster.coordination.http.replication.RequestReplicator;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.Revision;
import org.apache.nifi.web.api.dto.RevisionDTO;
import org.apache.nifi.web.api.entity.ComponentEntity;
import org.apache.nifi.web.api.entity.Entity;
import org.apache.nifi.web.api.request.ClientIdParameter;
import org.apache.nifi.web.security.jwt.JwtAuthenticationFilter;
import org.apache.nifi.web.util.WebUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.security.core.Authentication;
import org.springframework.security.core.context.SecurityContextHolder;
import com.sun.jersey.api.core.HttpContext;
import com.sun.jersey.api.representation.Form;
import com.sun.jersey.core.util.MultivaluedMapImpl;
import com.sun.jersey.server.impl.model.method.dispatch.FormDispatchProvider;
/**
* Base class for controllers.
@ -311,7 +316,7 @@ public abstract class ApplicationResource {
// get the form that jersey processed and use it if it exists (only exist for requests with a body and application form urlencoded
final Form form = (Form) httpContext.getProperties().get(FormDispatchProvider.FORM_PROPERTY);
if (form == null) {
for (Map.Entry<String, String[]> entry : (Set<Map.Entry<String, String[]>>) httpServletRequest.getParameterMap().entrySet()) {
for (Map.Entry<String, String[]> entry : httpServletRequest.getParameterMap().entrySet()) {
if (entry.getValue() == null) {
entity.add(entry.getKey(), null);
} else {
@ -407,4 +412,50 @@ public abstract class ApplicationResource {
strb.append(ReflectionToStringBuilder.toString(action, ToStringStyle.MULTI_LINE_STYLE)).append("\n");
}
}
/**
* Checks whether the request is part of a two-phase commit style request (either phase 1 or phase 2)
*
* @param httpServletRequest the request
* @return <code>true</code> if the request represents a two-phase commit style request
*/
protected boolean isTwoPhaseRequest(HttpServletRequest httpServletRequest) {
final String headerValue = httpServletRequest.getHeader(RequestReplicator.REQUEST_TRANSACTION_ID);
return headerValue != null;
}
/**
* When a two-phase commit style request is used, the first phase (generally referred to
* as the "commit-request stage") is intended to validate that the request can be completed.
* In NiFi, we use this phase to validate that the request can complete. This method determines
* whether or not the request is the first phase of a two-phase commit.
*
* @param httpServletRequest the request
* @return <code>true</code> if the request represents a two-phase commit style request and is the
* first of the two phases.
*/
protected boolean isValidationPhase(HttpServletRequest httpServletRequest) {
return isTwoPhaseRequest(httpServletRequest) && httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER) != null;
}
/**
* Converts a Revision DTO and an associated Component ID into a Revision object
*
* @param revisionDto the Revision DTO
* @param componentId the ID of the component that the Revision DTO belongs to
* @return a Revision that has the same client ID and Version as the Revision DTO and the Component ID specified
*/
protected Revision getRevision(RevisionDTO revisionDto, String componentId) {
return new Revision(revisionDto.getVersion(), revisionDto.getClientId(), componentId);
}
/**
* Extracts a Revision object from the Revision DTO and ID provided by the Component Entity
*
* @param entity the ComponentEntity that contains the Revision DTO & ID
* @return the Revision specified in the ComponentEntity
*/
protected Revision getRevision(ComponentEntity entity, String componentId) {
return getRevision(entity.getRevision(), componentId);
}
}

View File

@ -16,25 +16,10 @@
*/
package org.apache.nifi.web.api;
import com.wordnik.swagger.annotations.Api;
import com.wordnik.swagger.annotations.ApiOperation;
import com.wordnik.swagger.annotations.ApiParam;
import com.wordnik.swagger.annotations.ApiResponse;
import com.wordnik.swagger.annotations.ApiResponses;
import com.wordnik.swagger.annotations.Authorization;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.NiFiServiceFacade;
import org.apache.nifi.web.Revision;
import org.apache.nifi.web.UpdateResult;
import org.apache.nifi.web.api.dto.ConnectionDTO;
import org.apache.nifi.web.api.dto.FlowFileSummaryDTO;
import org.apache.nifi.web.api.dto.ListingRequestDTO;
import org.apache.nifi.web.api.dto.RevisionDTO;
import org.apache.nifi.web.api.entity.ConnectionEntity;
import org.apache.nifi.web.api.request.ClientIdParameter;
import org.apache.nifi.web.api.request.LongParameter;
import java.net.URI;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import javax.servlet.http.HttpServletRequest;
import javax.ws.rs.Consumes;
@ -50,10 +35,26 @@ import javax.ws.rs.QueryParam;
import javax.ws.rs.core.Context;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import java.net.URI;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.NiFiServiceFacade;
import org.apache.nifi.web.Revision;
import org.apache.nifi.web.UpdateResult;
import org.apache.nifi.web.api.dto.ConnectionDTO;
import org.apache.nifi.web.api.dto.FlowFileSummaryDTO;
import org.apache.nifi.web.api.dto.ListingRequestDTO;
import org.apache.nifi.web.api.entity.ConnectionEntity;
import org.apache.nifi.web.api.request.ClientIdParameter;
import org.apache.nifi.web.api.request.LongParameter;
import com.wordnik.swagger.annotations.Api;
import com.wordnik.swagger.annotations.ApiOperation;
import com.wordnik.swagger.annotations.ApiParam;
import com.wordnik.swagger.annotations.ApiResponse;
import com.wordnik.swagger.annotations.ApiResponses;
import com.wordnik.swagger.annotations.Authorization;
/**
* RESTful endpoint for managing a Connection.
@ -268,16 +269,20 @@ public class ConnectionResource extends ApplicationResource {
return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), updateClientId(connectionEntity), getHeaders(headersToOverride)).getResponse();
}
// handle expects request (usually from the cluster manager)
final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
if (expects != null) {
// handle expects request
final Revision revision = getRevision(connectionEntity, id);
final boolean validationPhase = isValidationPhase(httpServletRequest);
if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
serviceFacade.claimRevision(revision);
}
if (validationPhase) {
serviceFacade.verifyUpdateConnection(connection);
return generateContinueResponse().build();
}
// update the relationship target
final RevisionDTO revision = connectionEntity.getRevision();
final UpdateResult<ConnectionEntity> updateResult = serviceFacade.updateConnection(new Revision(revision.getVersion(), revision.getClientId()), connection);
final UpdateResult<ConnectionEntity> updateResult = serviceFacade.updateConnection(revision, connection);
final ConnectionEntity entity = updateResult.getResult();
populateRemainingConnectionEntityContent(entity);
@ -343,21 +348,23 @@ public class ConnectionResource extends ApplicationResource {
return clusterManager.applyRequest(HttpMethod.DELETE, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
}
// determine the specified version
final Long clientVersion = version == null ? null : version.getLong();
final Revision revision = new Revision(clientVersion, clientId.getClientId(), id);
// handle expects request (usually from the cluster manager)
final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
if (expects != null) {
final boolean validationPhase = isValidationPhase(httpServletRequest);
if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
serviceFacade.claimRevision(revision);
}
if (validationPhase) {
serviceFacade.verifyDeleteConnection(id);
return generateContinueResponse().build();
}
// determine the specified version
Long clientVersion = null;
if (version != null) {
clientVersion = version.getLong();
}
// delete the connection
final ConnectionEntity entity = serviceFacade.deleteConnection(new Revision(clientVersion, clientId.getClientId()), id);
final ConnectionEntity entity = serviceFacade.deleteConnection(revision, id);
// generate the response
return clusterContext(generateOkResponse(entity)).build();

View File

@ -155,20 +155,11 @@ public class ControllerResource extends ApplicationResource {
// create the archive
final RevisionDTO requestRevision = revisionEntity.getRevision();
final ConfigurationSnapshot<Void> controllerResponse = serviceFacade.createArchive(new Revision(requestRevision.getVersion(), requestRevision.getClientId()));
// create the revision
final RevisionDTO updatedRevision = new RevisionDTO();
updatedRevision.setClientId(requestRevision.getClientId());
updatedRevision.setVersion(controllerResponse.getVersion());
// create the response entity
final ProcessGroupEntity controllerEntity = new ProcessGroupEntity();
controllerEntity.setRevision(updatedRevision);
final ProcessGroupEntity entity = serviceFacade.createArchive(new Revision(requestRevision.getVersion(), requestRevision.getClientId()));
// generate the response
URI uri = URI.create(generateResourceUri("controller", "archive"));
return clusterContext(generateCreatedResponse(uri, controllerEntity)).build();
return clusterContext(generateCreatedResponse(uri, entity)).build();
}
/**
@ -432,15 +423,17 @@ public class ControllerResource extends ApplicationResource {
return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), updateClientId(configEntity), getHeaders()).getResponse();
}
final RevisionDTO revisionDto = configEntity.getRevision();
final Revision revision = new Revision(revisionDto.getVersion(), revisionDto.getClientId(), "controller");
// handle expects request (usually from the cluster manager)
final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
if (expects != null) {
return generateContinueResponse().build();
}
final RevisionDTO revision = configEntity.getRevision();
final ConfigurationSnapshot<ControllerConfigurationDTO> controllerResponse
= serviceFacade.updateControllerConfiguration(new Revision(revision.getVersion(), revision.getClientId()), configEntity.getConfig());
= serviceFacade.updateControllerConfiguration(revision, configEntity.getConfig());
final ControllerConfigurationDTO controllerConfig = controllerResponse.getConfiguration();
// get the updated revision
@ -580,9 +573,6 @@ public class ControllerResource extends ApplicationResource {
throw new IllegalArgumentException("The type of reporting task to create must be specified.");
}
// get the revision
final RevisionDTO revision = reportingTaskEntity.getRevision();
if (properties.isClusterManager()) {
return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), updateClientId(reportingTaskEntity), getHeaders()).getResponse();
}
@ -602,22 +592,13 @@ public class ControllerResource extends ApplicationResource {
}
// create the reporting task and generate the json
final ConfigurationSnapshot<ReportingTaskDTO> controllerResponse = serviceFacade.createReportingTask(
new Revision(revision.getVersion(), revision.getClientId()), reportingTaskEntity.getReportingTask());
final ReportingTaskDTO reportingTask = controllerResponse.getConfiguration();
final Revision revision = getRevision(reportingTaskEntity.getRevision(), reportingTaskEntity.getReportingTask().getId());
final ReportingTaskEntity entity = serviceFacade.createReportingTask(revision, reportingTaskEntity.getReportingTask());
// get the updated revision
final RevisionDTO updatedRevision = new RevisionDTO();
updatedRevision.setClientId(revision.getClientId());
updatedRevision.setVersion(controllerResponse.getVersion());
// build the response entity
final ReportingTaskEntity entity = new ReportingTaskEntity();
entity.setRevision(updatedRevision);
entity.setReportingTask(reportingTaskResource.populateRemainingReportingTaskContent(availability, reportingTask));
reportingTaskResource.populateRemainingReportingTaskContent(availability, entity.getReportingTask());
// build the response
return clusterContext(generateCreatedResponse(URI.create(reportingTask.getUri()), entity)).build();
return clusterContext(generateCreatedResponse(URI.create(entity.getReportingTask().getUri()), entity)).build();
}
/**

View File

@ -16,39 +16,11 @@
*/
package org.apache.nifi.web.api;
import com.wordnik.swagger.annotations.Api;
import com.wordnik.swagger.annotations.ApiOperation;
import com.wordnik.swagger.annotations.ApiParam;
import com.wordnik.swagger.annotations.ApiResponse;
import com.wordnik.swagger.annotations.ApiResponses;
import com.wordnik.swagger.annotations.Authorization;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import org.apache.nifi.controller.ScheduledState;
import org.apache.nifi.controller.service.ControllerServiceState;
import org.apache.nifi.ui.extension.UiExtension;
import org.apache.nifi.ui.extension.UiExtensionMapping;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.ConfigurationSnapshot;
import org.apache.nifi.web.NiFiServiceFacade;
import org.apache.nifi.web.Revision;
import org.apache.nifi.web.UiExtensionType;
import org.apache.nifi.web.api.dto.ComponentStateDTO;
import org.apache.nifi.web.api.dto.ControllerServiceDTO;
import org.apache.nifi.web.api.dto.ControllerServiceReferencingComponentDTO;
import org.apache.nifi.web.api.dto.PropertyDescriptorDTO;
import org.apache.nifi.web.api.dto.RevisionDTO;
import org.apache.nifi.web.api.entity.ComponentStateEntity;
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentsEntity;
import org.apache.nifi.web.api.entity.Entity;
import org.apache.nifi.web.api.entity.PropertyDescriptorEntity;
import org.apache.nifi.web.api.entity.UpdateControllerServiceReferenceRequestEntity;
import org.apache.nifi.web.api.request.ClientIdParameter;
import org.apache.nifi.web.api.request.LongParameter;
import org.apache.nifi.web.util.Availability;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.net.URI;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import javax.servlet.ServletContext;
import javax.servlet.http.HttpServletRequest;
@ -66,11 +38,41 @@ import javax.ws.rs.QueryParam;
import javax.ws.rs.core.Context;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import java.net.URI;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import org.apache.nifi.controller.ScheduledState;
import org.apache.nifi.controller.service.ControllerServiceState;
import org.apache.nifi.ui.extension.UiExtension;
import org.apache.nifi.ui.extension.UiExtensionMapping;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.ConfigurationSnapshot;
import org.apache.nifi.web.NiFiServiceFacade;
import org.apache.nifi.web.Revision;
import org.apache.nifi.web.UiExtensionType;
import org.apache.nifi.web.UpdateResult;
import org.apache.nifi.web.api.dto.ComponentStateDTO;
import org.apache.nifi.web.api.dto.ControllerServiceDTO;
import org.apache.nifi.web.api.dto.PropertyDescriptorDTO;
import org.apache.nifi.web.api.dto.RevisionDTO;
import org.apache.nifi.web.api.entity.ComponentStateEntity;
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentsEntity;
import org.apache.nifi.web.api.entity.Entity;
import org.apache.nifi.web.api.entity.PropertyDescriptorEntity;
import org.apache.nifi.web.api.entity.UpdateControllerServiceReferenceRequestEntity;
import org.apache.nifi.web.api.request.ClientIdParameter;
import org.apache.nifi.web.api.request.LongParameter;
import org.apache.nifi.web.util.Availability;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.wordnik.swagger.annotations.Api;
import com.wordnik.swagger.annotations.ApiOperation;
import com.wordnik.swagger.annotations.ApiParam;
import com.wordnik.swagger.annotations.ApiResponse;
import com.wordnik.swagger.annotations.ApiResponses;
import com.wordnik.swagger.annotations.Authorization;
/**
* RESTful endpoint for managing a Controller Service.
@ -402,19 +404,22 @@ public class ControllerServiceResource extends ApplicationResource {
}
// handle expects request (usually from the cluster manager)
final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
if (expects != null) {
final Revision revision = getRevision(revisionEntity.getRevision(), id);
final boolean validationPhase = isValidationPhase(httpServletRequest);
if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
serviceFacade.claimRevision(revision);
}
if (validationPhase) {
serviceFacade.verifyCanClearControllerServiceState(id);
return generateContinueResponse().build();
}
// get the component state
final RevisionDTO requestRevision = revisionEntity.getRevision();
final ConfigurationSnapshot<Void> snapshot = serviceFacade.clearControllerServiceState(new Revision(requestRevision.getVersion(), requestRevision.getClientId()), id);
final ConfigurationSnapshot<Void> snapshot = serviceFacade.clearControllerServiceState(revision, id);
// create the revision
final RevisionDTO responseRevision = new RevisionDTO();
responseRevision.setClientId(requestRevision.getClientId());
responseRevision.setClientId(revision.getClientId());
responseRevision.setVersion(snapshot.getVersion());
// generate the response entity
@ -478,11 +483,7 @@ public class ControllerServiceResource extends ApplicationResource {
}
// get the controller service
final Set<ControllerServiceReferencingComponentDTO> controllerServiceReferences = serviceFacade.getControllerServiceReferencingComponents(id);
// create the response entity
final ControllerServiceReferencingComponentsEntity entity = new ControllerServiceReferencingComponentsEntity();
entity.setControllerServiceReferencingComponents(controllerServiceReferences);
final ControllerServiceReferencingComponentsEntity entity = serviceFacade.getControllerServiceReferencingComponents(id);
return clusterContext(generateOkResponse(entity)).build();
}
@ -574,26 +575,19 @@ public class ControllerServiceResource extends ApplicationResource {
}
// handle expects request (usually from the cluster manager)
final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
if (expects != null) {
final Revision controllerServiceRevision = getRevision(updateReferenceRequest.getRevision(), updateReferenceRequest.getId());
final boolean validationPhase = isValidationPhase(httpServletRequest);
if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
serviceFacade.claimRevision(controllerServiceRevision);
}
if (validationPhase) {
serviceFacade.verifyUpdateControllerServiceReferencingComponents(updateReferenceRequest.getId(), scheduledState, controllerServiceState);
return generateContinueResponse().build();
}
// get the controller service
final RevisionDTO requestRevision = updateReferenceRequest.getRevision();
final ConfigurationSnapshot<Set<ControllerServiceReferencingComponentDTO>> response = serviceFacade.updateControllerServiceReferencingComponents(
new Revision(requestRevision.getVersion(), requestRevision.getClientId()), updateReferenceRequest.getId(), scheduledState, controllerServiceState);
// create the revision
final RevisionDTO revision = new RevisionDTO();
revision.setClientId(requestRevision.getClientId());
revision.setVersion(response.getVersion());
// create the response entity
final ControllerServiceReferencingComponentsEntity entity = new ControllerServiceReferencingComponentsEntity();
entity.setRevision(revision);
entity.setControllerServiceReferencingComponents(response.getConfiguration());
final ControllerServiceReferencingComponentsEntity entity = serviceFacade.updateControllerServiceReferencingComponents(
controllerServiceRevision, updateReferenceRequest.getId(), scheduledState, controllerServiceState);
return clusterContext(generateOkResponse(entity)).build();
}
@ -676,32 +670,25 @@ public class ControllerServiceResource extends ApplicationResource {
}
// handle expects request (usually from the cluster manager)
final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
if (expects != null) {
final Revision revision = getRevision(controllerServiceEntity, id);
final boolean validationPhase = isValidationPhase(httpServletRequest);
if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
serviceFacade.claimRevision(revision);
}
if (validationPhase) {
serviceFacade.verifyUpdateControllerService(requestControllerServiceDTO);
return generateContinueResponse().build();
}
// update the controller service
final RevisionDTO revision = controllerServiceEntity.getRevision();
final ConfigurationSnapshot<ControllerServiceDTO> controllerResponse = serviceFacade.updateControllerService(
new Revision(revision.getVersion(), revision.getClientId()), requestControllerServiceDTO);
// get the results
final ControllerServiceDTO responseControllerServiceDTO = controllerResponse.getConfiguration();
// get the updated revision
final RevisionDTO updatedRevision = new RevisionDTO();
updatedRevision.setClientId(revision.getClientId());
updatedRevision.setVersion(controllerResponse.getVersion());
final UpdateResult<ControllerServiceEntity> updateResult = serviceFacade.updateControllerService(revision, requestControllerServiceDTO);
// build the response entity
final ControllerServiceEntity entity = new ControllerServiceEntity();
entity.setRevision(updatedRevision);
entity.setControllerService(populateRemainingControllerServiceContent(availability, responseControllerServiceDTO));
final ControllerServiceEntity entity = updateResult.getResult();
populateRemainingControllerServiceContent(availability, entity.getControllerService());
if (controllerResponse.isNew()) {
return clusterContext(generateCreatedResponse(URI.create(responseControllerServiceDTO.getUri()), entity)).build();
if (updateResult.isNew()) {
return clusterContext(generateCreatedResponse(URI.create(entity.getControllerService().getUri()), entity)).build();
} else {
return clusterContext(generateOkResponse(entity)).build();
}
@ -775,30 +762,18 @@ public class ControllerServiceResource extends ApplicationResource {
}
// handle expects request (usually from the cluster manager)
final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
if (expects != null) {
final Revision revision = new Revision(version == null ? null : version.getLong(), clientId.getClientId(), id);
final boolean validationPhase = isValidationPhase(httpServletRequest);
if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
serviceFacade.claimRevision(revision);
}
if (validationPhase) {
serviceFacade.verifyDeleteControllerService(id);
return generateContinueResponse().build();
}
// determine the specified version
Long clientVersion = null;
if (version != null) {
clientVersion = version.getLong();
}
// delete the specified controller service
final ConfigurationSnapshot<Void> controllerResponse = serviceFacade.deleteControllerService(new Revision(clientVersion, clientId.getClientId()), id);
// get the updated revision
final RevisionDTO revision = new RevisionDTO();
revision.setClientId(clientId.getClientId());
revision.setVersion(controllerResponse.getVersion());
// build the response entity
final ControllerServiceEntity entity = new ControllerServiceEntity();
entity.setRevision(revision);
final ControllerServiceEntity entity = serviceFacade.deleteControllerService(revision, id);
return clusterContext(generateOkResponse(entity)).build();
}

View File

@ -77,6 +77,7 @@ import java.util.UUID;
value = "/flowfile-queues",
description = "Endpoint for managing a FlowFile Queue."
)
// TODO: Need revisions of the Connections for these endpoints!
public class FlowFileQueueResource extends ApplicationResource {
private NiFiServiceFacade serviceFacade;

View File

@ -357,6 +357,10 @@ public class FlowResource extends ApplicationResource {
public Response getRevision() {
authorizeFlow();
if (properties.isClusterManager()) {
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
}
// create the current revision
final RevisionDTO revision = serviceFacade.getRevision();

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