NIFI-259: Initial implementation of State Management feature

This commit is contained in:
Mark Payne 2016-01-11 08:28:12 -05:00
parent 0c68e2c3a8
commit 57dadb7286
130 changed files with 4241 additions and 827 deletions

View File

@ -0,0 +1,36 @@
/*
* 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.components.state;
/**
* A Scope represents how a NiFi component's state is to be stored and retrieved when running in a cluster.
*/
public enum Scope {
/**
* State is to be treated as "global" across the cluster. I.e., the same component on all nodes will
* have access to the same state.
*/
CLUSTER,
/**
* State is to be treated local to the node. I.e., the same component will have different state on each
* node in the cluster.
*/
LOCAL;
}

View File

@ -0,0 +1,84 @@
/*
* 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.components.state;
import java.io.IOException;
import java.util.Map;
/**
* <p>
* The StateManager is responsible for providing NiFi components a mechanism for storing
* and retrieving state.
* </p>
*
* <p>
* When calling methods in this class, the {@link Scope} is used in order to specify whether
* state should be stored/retrieved from the local state or the clustered state. However, if
* any instance of NiFi is not clustered (or is disconnected from its cluster), the Scope is
* not really relevant and the local state will be used in all cases. This allows component
* developers to not concern themselves with whether or not a particular instance of NiFi is
* clustered. Instead, developers should assume that the instance is indeed clustered and write
* the component accordingly. If not clustered, the component will still behavior in the same
* manner, as a standalone node could be thought of as a "cluster of 1."
* </p>
*/
public interface StateManager {
/**
* Updates the value of the component's state, setting it to given value
*
* @param state the value to change the state to
* @param scope the scope to use when storing the state
*
* @throws IOException if unable to communicate with the underlying storage mechanism
*/
void setState(Map<String, String> state, Scope scope) throws IOException;
/**
* Returns the current state for the component. This return value will never be <code>null</code>.
* If the state has not yet been set, the StateMap's version will be -1, and the map of values will be empty.
*
* @param scope the scope to use when fetching the state
* @return
* @throws IOException
*/
StateMap getState(Scope scope) throws IOException;
/**
* Updates the value of the component's state to the new value if and only if the value currently
* is the same as the given oldValue.
*
* @param oldValue the old value to compare against
* @param newValue the new value to use if and only if the state's value is the same as the given oldValue
* @param scope the scope to use for storing the new state
* @return <code>true</code> if the state was updated to the new value, <code>false</code> if the state's value was not
* equal to oldValue
*
* @throws IOException if unable to communicate with the underlying storage mechanism
*/
boolean replace(StateMap oldValue, Map<String, String> newValue, Scope scope) throws IOException;
/**
* Clears all keys and values from the component's state
*
* @param scope the scope whose values should be cleared
*
* @throws IOException if unable to communicate with the underlying storage mechanism
*/
void clear(Scope scope) throws IOException;
}

View File

@ -0,0 +1,51 @@
/*
* 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.components.state;
import java.util.Map;
/**
* Provides a representation of a component's state at some point in time.
*/
public interface StateMap {
/**
* Each time that a component's state is updated, the state is assigned a new version.
* This version can then be used to atomically update state by the backing storage mechanism.
* Though this number is monotonically increasing, it should not be expected to increment always
* from X to X+1. I.e., version numbers may be skipped.
*
* @return the version associated with the state
*/
long getVersion();
/**
* Returns the value associated with the given key
*
* @param key the key whose value should be retrieved
* @return the value associated with the given key, or <code>null</code> if no value is associated
* with this key.
*/
String get(String key);
/**
* Returns an immutable Map representation of all keys and values for the state of a component.
*
* @return an immutable Map representation of all keys and values for the state of a component.
*/
Map<String, String> toMap();
}

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.
*/
package org.apache.nifi.components.state;
import java.io.IOException;
import java.util.Map;
import org.apache.nifi.components.ConfigurableComponent;
/**
* <p>
* Provides a mechanism by which components can store and retrieve state. Depending on the Provider, the state
* may be stored locally, or it may be stored on a remote resource.
* </p>
*
* <p>
* Which implementation should be used for local and clustered state is configured in the NiFi properties file.
* It is therefore possible to provide custom implementations of this interface. Note, however, that this interface
* is new as of version 0.5.0 of Apache NiFi and may not be considered "stable" as of yet. Therefore, it is subject
* to change without notice, so providing custom implementations is cautioned against until the API becomes more stable.
* </p>
*
* @since 0.5.0
*/
public interface StateProvider extends ConfigurableComponent {
/**
* Initializes the StateProvider so that it is capable of being used. This method will be called
* once before any of the other methods are called and will not be called again until the {@link #shutdown()}
* method has been called
*
* @param context the initialization context that can be used to prepare the state provider for use
*/
void initialize(StateProviderInitializationContext context) throws IOException;
/**
* Shuts down the StateProvider and cleans up any resources held by it. Once this method has returned, the
* StateProvider may be initialized once again via the {@link #initialize(StateProviderInitializationContext)} method.
*/
void shutdown();
/**
* Updates the value of the component's state, setting the new value to the
* given state
*
* @param state the value to change the state to
* @param componentId the id of the component for which state is being set
*
* @throws IOException if unable to communicate with the underlying storage mechanism
*/
void setState(Map<String, String> state, String componentId) throws IOException;
/**
* Returns the currently configured state for the component. The returned StateMap will never be null.
* The version of the StateMap will be -1 and the state will contain no key/value pairs if the state has never been set.
*
* @param componentId the id of the component for which state is to be retrieved
* @return the currently configured value for the component's state
*
* @throws IOException if unable to communicate with the underlying storage mechanism
*/
StateMap getState(String componentId) throws IOException;
/**
* Updates the value of the component's state to the new value if and only if the value currently
* is the same as the given oldValue.
*
* @param oldValue the old value to compare against
* @param newValue the new value to use if and only if the state's value is the same as the given oldValue
* @param componentId the id of the component for which state is being retrieved
* @return <code>true</code> if the state was updated to the new value, <code>false</code> if the state's value was not
* equal to oldValue
*
* @throws IOException if unable to communicate with the underlying storage mechanism
*/
boolean replace(StateMap oldValue, Map<String, String> newValue, String componentId) throws IOException;
/**
* Removes all values from the component's state that is stored using the given scope
*
* @param componentId the id of the component for which state is being cleared
*
* @throws IOException if unable to communicate with the underlying storage mechanism
*/
void clear(String componentId) throws IOException;
/**
* This method is called whenever a component is removed from the NiFi instance. This allows the State Provider to
* perform tasks when a component is removed in order to clean up resources that may be associated with that component
*
* @param componentId the ID of the component that was added to the NiFi instance
* @throws IOException if unable to perform the necessary cleanup
*/
void onComponentRemoved(String componentId) throws IOException;
/**
* Notifies the state provider that it should begin servicing requests to store and retrieve state
*/
void enable();
/**
* Notifies the state provider that it should stop servicing requests to store and retrieve state and instead throw a ProviderDisabledException if any request is made to do so
*/
void disable();
/**
* @return <code>true</code> if the provider is enabled, <code>false</code> otherwise.
*/
boolean isEnabled();
}

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.components.state;
import java.util.Map;
import javax.net.ssl.SSLContext;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue;
/**
* This interface defines an initialization context that is passed to a {@link StateProvider} when it
* is initialized.
*/
public interface StateProviderInitializationContext {
/**
* @return the identifier if the StateProvider
*/
String getIdentifier();
/**
* @return a Map of Property Descriptors to their configured values
*/
Map<PropertyDescriptor, PropertyValue> getProperties();
/**
* Returns the configured value for the given property
*
* @param property the property to retrieve the value for
*
* @return the configured value for the property.
*/
PropertyValue getProperty(PropertyDescriptor property);
/**
* @return the SSL Context that should be used to communicate with remote resources,
* or <code>null</code> if no SSLContext has been configured
*/
SSLContext getSSLContext();
}

View File

@ -16,6 +16,7 @@
*/
package org.apache.nifi.controller;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.logging.ComponentLog;
public interface ControllerServiceInitializationContext {
@ -37,4 +38,9 @@ public interface ControllerServiceInitializationContext {
* way and generate bulletins when appropriate
*/
ComponentLog getLogger();
/**
* @return the StateManager that can be used to store and retrieve state for this component
*/
StateManager getStateManager();
}

View File

@ -21,6 +21,7 @@ import java.util.Set;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.controller.ControllerServiceLookup;
/**
@ -154,4 +155,9 @@ public interface ProcessContext {
* does not allow the Expression Language, even if a seemingly valid Expression is present in the value.
*/
boolean isExpressionLanguagePresent(PropertyDescriptor property);
/**
* @return the StateManager that can be used to store and retrieve state for this component
*/
StateManager getStateManager();
}

View File

@ -25,6 +25,7 @@ import java.util.Map;
import java.util.Set;
import java.util.regex.Pattern;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.controller.queue.QueueSize;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.exception.FlowFileAccessException;

View File

@ -20,6 +20,7 @@ import java.util.Map;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.controller.ControllerServiceLookup;
/**
@ -86,4 +87,9 @@ public interface ReportingContext {
* Controller Services
*/
ControllerServiceLookup getControllerServiceLookup();
/**
* @return the StateManager that can be used to store and retrieve state for this component
*/
StateManager getStateManager();
}

View File

@ -283,6 +283,12 @@ language governing permissions and limitations under the License. -->
<nifi.templates.directory>./conf/templates</nifi.templates.directory>
<nifi.database.directory>./database_repository</nifi.database.directory>
<nifi.state.management.configuration.file>./conf/state-management.xml</nifi.state.management.configuration.file>
<nifi.state.management.embedded.zookeeper.start>false</nifi.state.management.embedded.zookeeper.start>
<nifi.state.management.embedded.zookeeper.properties>./conf/zookeeper.properties</nifi.state.management.embedded.zookeeper.properties>
<nifi.state.management.provider.local>local-provider</nifi.state.management.provider.local>
<nifi.state.management.provider.cluster>zk-provider</nifi.state.management.provider.cluster>
<nifi.flowfile.repository.implementation>org.apache.nifi.controller.repository.WriteAheadFlowFileRepository</nifi.flowfile.repository.implementation>
<nifi.flowfile.repository.directory>./flowfile_repository</nifi.flowfile.repository.directory>
<nifi.flowfile.repository.partitions>256</nifi.flowfile.repository.partitions>

View File

@ -189,6 +189,13 @@ public class NiFiProperties extends Properties {
// kerberos properties
public static final String KERBEROS_KRB5_FILE = "nifi.kerberos.krb5.file";
// state management
public static final String STATE_MANAGEMENT_CONFIG_FILE = "nifi.state.management.configuration.file";
public static final String STATE_MANAGEMENT_LOCAL_PROVIDER_ID = "nifi.state.management.provider.local";
public static final String STATE_MANAGEMENT_CLUSTER_PROVIDER_ID = "nifi.state.management.provider.cluster";
public static final String STATE_MANAGEMENT_START_EMBEDDED_ZOOKEEPER = "nifi.state.management.embedded.zookeeper.start";
public static final String STATE_MANAGEMENT_ZOOKEEPER_PROPERTIES = "nifi.state.management.embedded.zookeeper.properties";
// defaults
public static final String DEFAULT_TITLE = "NiFi";
public static final Boolean DEFAULT_AUTO_RESUME_STATE = true;
@ -236,6 +243,9 @@ public class NiFiProperties extends Properties {
public static final int DEFAULT_CLUSTER_MANAGER_PROTOCOL_THREADS = 10;
public static final String DEFAULT_CLUSTER_MANAGER_SAFEMODE_DURATION = "0 sec";
// state management defaults
public static final String DEFAULT_STATE_MANAGEMENT_CONFIG_FILE = "conf/state-management.xml";
private NiFiProperties() {
super();
}
@ -985,4 +995,28 @@ public class NiFiProperties extends Properties {
public String getBoredYieldDuration() {
return getProperty(BORED_YIELD_DURATION, DEFAULT_BORED_YIELD_DURATION);
}
public File getStateManagementConfigFile() {
return new File(getProperty(STATE_MANAGEMENT_CONFIG_FILE, DEFAULT_STATE_MANAGEMENT_CONFIG_FILE));
}
/*
* public static final String STATE_MANAGEMENT_MAX_ZOOKEEPER_SERVERS = "nifi.state.management.embedded.zookeeper.max.instances";
*/
public String getLocalStateProviderId() {
return getProperty(STATE_MANAGEMENT_LOCAL_PROVIDER_ID);
}
public String getClusterStateProviderId() {
return getProperty(STATE_MANAGEMENT_CLUSTER_PROVIDER_ID);
}
public File getEmbeddedZooKeeperPropertiesFile() {
final String filename = getProperty(STATE_MANAGEMENT_ZOOKEEPER_PROPERTIES);
return filename == null ? null : new File(filename);
}
public boolean isStartEmbeddedZooKeeper() {
return Boolean.parseBoolean(getProperty(STATE_MANAGEMENT_START_EMBEDDED_ZOOKEEPER));
}
}

View File

@ -761,7 +761,7 @@ public class EndpointConnectionPool {
final int index = n % destinations.size();
PeerStatus status = destinations.get(index);
if (status == null) {
final PeerDescription description = new PeerDescription(nodeInfo.getHostname(), nodeInfo.getSiteToSitePort(), nodeInfo.isSiteToSiteSecure());
final PeerDescription description = new PeerDescription(nodeInfo.getSiteToSiteHostname(), nodeInfo.getSiteToSitePort(), nodeInfo.isSiteToSiteSecure());
status = new PeerStatus(description, nodeInfo.getTotalFlowFiles());
destinations.set(index, status);
break;

View File

@ -18,23 +18,23 @@ package org.apache.nifi.remote.cluster;
public class NodeInformation {
private final String hostname;
private final String siteToSiteHostname;
private final Integer siteToSitePort;
private final int apiPort;
private final boolean isSiteToSiteSecure;
private final int totalFlowFiles;
public NodeInformation(final String hostname, final Integer siteToSitePort, final int apiPort,
public NodeInformation(final String siteToSiteHostname, final Integer siteToSitePort, final int apiPort,
final boolean isSiteToSiteSecure, final int totalFlowFiles) {
this.hostname = hostname;
this.siteToSiteHostname = siteToSiteHostname;
this.siteToSitePort = siteToSitePort;
this.apiPort = apiPort;
this.isSiteToSiteSecure = isSiteToSiteSecure;
this.totalFlowFiles = totalFlowFiles;
}
public String getHostname() {
return hostname;
public String getSiteToSiteHostname() {
return siteToSiteHostname;
}
public int getAPIPort() {
@ -66,7 +66,7 @@ public class NodeInformation {
}
final NodeInformation other = (NodeInformation) obj;
if (!hostname.equals(other.hostname)) {
if (!siteToSiteHostname.equals(other.siteToSiteHostname)) {
return false;
}
if (siteToSitePort == null && other.siteToSitePort != null) {
@ -88,11 +88,11 @@ public class NodeInformation {
@Override
public int hashCode() {
return 83832 + hostname.hashCode() + (siteToSitePort == null ? 8 : siteToSitePort.hashCode()) + apiPort + (isSiteToSiteSecure ? 3829 : 0);
return 83832 + siteToSiteHostname.hashCode() + (siteToSitePort == null ? 8 : siteToSitePort.hashCode()) + apiPort + (isSiteToSiteSecure ? 3829 : 0);
}
@Override
public String toString() {
return "Node[" + hostname + ":" + apiPort + "]";
return "Node[" + siteToSiteHostname + ":" + apiPort + "]";
}
}

View File

@ -30,7 +30,7 @@ public class NodeInformationAdapter extends XmlAdapter<AdaptedNodeInformation, N
@Override
public AdaptedNodeInformation marshal(final NodeInformation nodeInformation) throws Exception {
final AdaptedNodeInformation adapted = new AdaptedNodeInformation();
adapted.setHostname(nodeInformation.getHostname());
adapted.setHostname(nodeInformation.getSiteToSiteHostname());
adapted.setSiteToSitePort(nodeInformation.getSiteToSitePort());
adapted.setApiPort(nodeInformation.getAPIPort());
adapted.setSiteToSiteSecure(nodeInformation.isSiteToSiteSecure());

View File

@ -367,6 +367,78 @@ in the remote cluster can be included in the same group. When the ADMIN wants to
cluster, s/he can grant it to the group and avoid having to grant it individually to each node in the cluster.
[[state_management]]
State Management
----------------
NiFi provides a mechanism for Processors, Reporting Tasks, Controller Services, and the framework itself to persist state. This
allows a Processor, for example, to resume from the place where it left off after NiFi is restarted. Additionally, it allows for
a Processor to store some piece of information so that the Processor can access that information from all of the different nodes
in the cluster. This allows one node to pick up where another node left off, or to coordinate across all of the nodes in a cluster.
[[state_providers]]
*Configuring State Providers*
When a component decides to store or retrieve state, it does so by providing a "Scope" - either Node-local or Cluster-wide. The
mechanism that is used to store and retrieve this state is then determined based on this Scope, as well as the configured State
Providers. The _nifi.properties_ file contains three different properties that are relevant to configuring these State Providers.
The first is the `nifi.state.management.configuration.file` property specifies an external XML file that is used for configuring
the local and cluster-wide State Providers. This XML file may contain configurations for multiple providers, so the
`nifi.state.management.provider.local` property provides the identifier of the local State Provider configured in this XML file.
Similarly, the `nifi.state.management.provider.cluster` property provides the identifier of the cluster-wide State Provider
configured in this XML file.
This XML file consists of a top-level `state-management` element, which has one or more `local-provider` and zero or more
`cluster-provider` elements. Each of these elements then contains an `id` element that is used to specify the identifier that can
be referenced in the _nifi.properties_ file, as well as a `class` element that specifies the fully-qualified class name to use
in order to instantiate the State Provider. Finally, each of these elements may have zero or more `property` elements. Each
`property` element has an attribute, `name` that is the name of the property that the State Provider supports. The textual content
of the `property` element is the value of the property.
Once these State Providers have been configured in the _state-management.xml_ file (or whatever file is configured), those Providers
may be referenced by their identifiers. By default, the Local State Provider is configured to be a `WriteAheadLocalStateProvider` that
persists the data to the _$NIFI_HOME/state_ directory. The default Cluster State Provider is configured to be a `ZooKeeperStateProvider`.
The default ZooKeeper-based provider must have its `Connect String` property populated before it can be used. It is also advisable,
if multiple NiFi instances will use the same ZooKeeper instance, that the value of the `Root Node` property be changed. For instance,
one might set the value to `/nifi/<team name>/production`.
If NiFi is configured to run in a standalone mode, the `cluster-state-provider` element need not be populated in the _state-management.xml_
file and will actually be ignored if they are populated. However, the `local-state-provider` element must always be present and populated.
Additionally, if NiFi is run in a cluster, each node must also have the `cluster-state-provider` element present and properly configured. Otherwise,
NiFi will fail to startup.
While there are not many properties that need to be configured for these providers, they were externalized into a separate _state-providers.xml_
file, rather than being configured via the _nifi.properties_ file, simply because different implementations may require different properties,
and it is easier to maintain and understand the configuration in an XML-based file such as this, than to mix the properties of the Provider
in with all of the other NiFi framework-specific properties.
[[embedded_zookeeper]]
*Embedded ZooKeeper Server*
As mentioned above, the default State Provider for cluster-wide state is the `ZooKeeperStateProvider`. At the time of this writing, this is the
only State Provider that exists for handling cluster-wide state. What this means is that NiFi has a dependencies on ZooKeeper in order to
behave as a cluster. However, there are many environments in which NiFi is deployed where there is no existing ZooKeeper ensemble being maintained.
In order to avoid the burden of forcing administrators to also maintain a separate ZooKeeper instance, NiFi provides the option of starting an
embedded ZooKeeper server.
This can be accomplished by setting the `nifi.state.management.embedded.zookeeper.start` property in _nifi.properties_ to `true` on those nodes
that should run the embedded ZooKeeper server. Generally, it is advisable to run ZooKeeper on either 3 or 5 nodes. Running on fewer than 3 nodes
provides less durability in the face of failure. Running on more than 5 nodes generally produces more network traffic than is necessary. Additionally,
running ZooKeeper on 4 nodes provides no more benefit than running on 3 nodes, ZooKeeper requires a majority of nodes be active in order to function.
However, it is up to the administrator to determine the number of nodes most appropriate to the particular deployment of NiFi.
If the `nifi.state.management.embedded.zookeeper.start` property is set to `true`, the `nifi.state.management.embedded.zookeeper.properties` property
in _nifi.properties_ also becomes relevant. This specifies the ZooKeeper properties file to use. At a minimum, This properties file needs to be populated
with the list of ZooKeeper servers. Each of these servers is configured as <hostname>:<client port>[:<leader election port>]. For example, `myhost:2888:3888`.
This list of nodes should be the same nodes in the NiFi cluster that have the `nifi.state.management.embedded.zookeeper.start`
property set to `true`. Also note that because ZooKeeper will be listening on these ports, the firewall may need to be configured to open these ports
for incoming traffic, at least between nodes in the cluster.
For more information on the properties used to administer ZooKeeper, see the
link:https://zookeeper.apache.org/doc/current/zookeeperAdmin.html[ZooKeeper Admin Guide].
[[clustering]]
Clustering Configuration
------------------------
@ -433,6 +505,7 @@ For the NCM, the minimum properties to configure are as follows:
For Node 1, the minimum properties to configure are as follows:
* Under the Web Properties, set either the http or https port that you want Node 1 to run on. If the NCM is running on the same server, choose a different web port for Node 1. Also, consider whether you need to set the http or https host property.
* Under the State Management section, set the `nifi.state.management.provider.cluster` property to the identifier of the Cluster State Provider. Ensure that the Cluster State Provider has been configured in the _state-management.xml_ file. See <<state_providers>> for more information.
* Under Cluster Node Properties, set the following:
** nifi.cluster.is.node - Set this to _true_.
** nifi.cluster.node.address - Set this to the fully qualified hostname of the node. If left blank, it defaults to "localhost".
@ -443,6 +516,7 @@ For Node 1, the minimum properties to configure are as follows:
For Node 2, the minimum properties to configure are as follows:
* Under the Web Properties, set either the http or https port that you want Node 2 to run on. Also, consider whether you need to set the http or https host property.
* Under the State Management section, set the `nifi.state.management.provider.cluster` property to the identifier of the Cluster State Provider. Ensure that the Cluster State Provider has been configured in the _state-management.xml_ file. See <<state_providers>> for more information.
* Under the Cluster Node Properties, set the following:
** nifi.cluster.is.node - Set this to _true_.
** nifi.cluster.node.address - Set this to the fully qualified hostname of the node. If left blank, it defaults to "localhost".
@ -631,6 +705,22 @@ only consider if `nifi.security.user.login.identity.provider` configured with a
|nifi.documentation.working.directory|The documentation working directory. The default value is ./work/docs/components and probably should be left as is.
|====
*State Management* +
The State Management section of the Properties file provides a mechanism for configuring local and cluster-wide mechanisms
for components to persist state. See the <<state_management>> section for more information on how this is used.
|====
|*Property*|*Description*
|nifi.state.management.configuration.file|The XML file that contains configuration for the local and cluster-wide State Providers. The default value is _./conf/state-management.xml_
|nifi.state.management.provider.local|The ID of the Local State Provider to use. This value must match the value of the `id` element of one of the `local-provider` elements in the _state-management.xml_ file.
|nifi.state.management.provider.cluster|The ID of the Cluster State Provider to use. This value must match the value of the `id` element of one of the `cluster-provider` elements in the _state-management.xml_ file. This value is ignored if not clustered but is required for nodes in a cluster.
|nifi.state.management.embedded.zookeeper.start|Specifies whether or not this instance of NiFi should start an embedded ZooKeeper Server. This is used in conjunction with the ZooKeeperStateProvider.
|nifi.state.management.embedded.zookeeper.properties|Specifies a properties file that contains the configuration for the embedded ZooKeeper Server that is started (if the `|nifi.state.management.embedded.zookeeper.start` property is set to `true`)
||====
*H2 Settings* +
The H2 Settings section defines the settings for the H2 database, which keeps track of user access and flow controller history.

View File

@ -216,6 +216,13 @@ class, calling the appropriate methods
to fill in the details of the Relationship, and finally calling the
`build` method.
[[state_manager]]
==== StateManager
The StateManager provides Processors, Reporting Tasks, and Controller Services a mechanism
for easily storing and retrieving state. The API is similar to that of ConcurrentHashMap
but requires a Scope for each operation. The Scope indicates whether the state is to be
retrieved/stored locally or in a cluster-wide manner. For more information, see the
<<state_manager>> section.
[[processor_initialization_context]]
==== ProcessorInitializationContext
@ -558,6 +565,64 @@ for instance, they should not be
relied upon for critical business logic.
[[state_manager]]
=== State Manager
From the ProcessContext, ReportingContext, and ControllerServiceInitializationContext, components are
able to call the `getStateManager()` method. This State Manager is responsible for providing a simple API
for storing and retrieving state. As such, the API is designed to be quite similar to the ConcurrentMap
API, which most Java developers are already familiar with.
[[state_scope]]
==== Scope
One very notable difference between the StateManager API and the ConcurrentMap API, however, is the presence
of a Scope object on each method call of the StateManager. This Scope will either be `Scope.NODE` or `Scope.CLUSTER`.
If NiFi is run in a cluster, this Scope provides important information to the framework about how the operation should
occur.
If state as stored using `Scope.CLUSTER`, then all nodes in the cluster will be communicating with the same
state storage mechanism, as if all nodes were to share a single ConcurrentMap. If state is stored and retrieved using
`Scope.NODE`, then each node will see a different representation of the state.
It is also worth noting that if NiFi is configured to run as a standalone instance, rather than running in a cluster,
a scope of `Scope.NODE` is always used. This is done in order to allow the developer of a NiFi component to write the code
in one consistent way, without worrying about whether or not the NiFi instance is clustered. The developer should instead assume
that the instance is clustered and write the code accordingly.
==== Storing and Retrieving State
State is stored using the StateManager's `set`, `replace`, `putIfAbsent`, `remove`, and `clear` methods. All of these methods,
with the exception of `clear` take as the first argument the key to be set. The key that is used is unique only to the same
instance of the component and for the same Scope. That is, if two Processors store a value using the key _My Key_, those Processors
will not conflict with each other, even if both Processors are of the same type (e.g., both are of type ListFile). Furthermore,
if a Processor stores a value with the key of _My Key_ using the `Scope.CLUSTER` scope, and then attempts to retrieve the value
using the `Scope.NODE` scope, the value retrieved will be `null`. Each Processor's state, then, is stored in isolation from other
Processor's state. A unique key can be thought of as a triple of <Processor Instance, Key, Scope>.
It follows, then, that two Processors cannot share the same state. There are, however, some circumstances in which it is very
necessary to share state between two Processors of different types, or two Processors of the same type. This can be accomplished
by using a Controller Service. By storing and retrieving state from a Controller Service, multiple Processors can use the same
Controller Service and the state can be exposed via the Controller Service's API.
==== Unit Tests
NiFi's Mock Framework provides an extensive collection of tools to perform unit testing of Processors. Processor unit tests typically
begin with the `TestRunner` class. As a result, the `TestRunner` class contains a `getStateManager` method of its own. The StateManager
that is returned, however, is of a specific type: `MockStateManager`. This implementation provides several methods in addition to those
defined by the `StateManager` interface, that help developers to more easily develop unit tests.
First, the `MockStateManager` implements the `StateManager` interface, so all of the state can be examined from within a unit test.
Additionally, the `MockStateManager` exposes a handful of `assert*` methods to perform assertions that the State is set as expected.
There are times, however, that state could be updated multiple times during the run of a single invocation of a Processor's `onTrigger`
method. In this case, inspecting the values after running the Processor may not be sufficient. Additionally, we must always remember at each
step to check the value of the stored state, which can become error-prone and burdensome for the developer.
For these reasons, the `MockStateManager` provides an additional method, named `failIfStateSet`. This method instructs the State Manager that
the unit test should immediately fail if the state for a given key is ever set, or if it is set to a specific value. The `doNotFailIfStateSet`
method can then be used to instruct the Mock Framework to clear this state and allow state to be set to any value.
=== Reporting Processor Activity

View File

@ -0,0 +1,172 @@
/*
* 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.state;
import java.util.Collections;
import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.nifi.components.state.Scope;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.components.state.StateMap;
import org.junit.Assert;
public class MockStateManager implements StateManager {
private final AtomicInteger versionIndex = new AtomicInteger(0);
private StateMap localStateMap = new MockStateMap(null, -1L);
private StateMap clusterStateMap = new MockStateMap(null, -1L);
@Override
public synchronized void setState(final Map<String, String> state, final Scope scope) {
final StateMap stateMap = new MockStateMap(state, versionIndex.incrementAndGet());
if (scope == Scope.CLUSTER) {
clusterStateMap = stateMap;
} else {
localStateMap = stateMap;
}
}
@Override
public synchronized StateMap getState(final Scope scope) {
if (scope == Scope.CLUSTER) {
return clusterStateMap;
} else {
return localStateMap;
}
}
@Override
public synchronized boolean replace(final StateMap oldValue, final Map<String, String> newValue, final Scope scope) {
if (scope == Scope.CLUSTER) {
if (oldValue == clusterStateMap) {
clusterStateMap = new MockStateMap(newValue, versionIndex.incrementAndGet());
return true;
}
return false;
} else {
if (oldValue == localStateMap) {
localStateMap = new MockStateMap(newValue, versionIndex.incrementAndGet());
return true;
}
return false;
}
}
@Override
public synchronized void clear(final Scope scope) {
setState(Collections.<String, String> emptyMap(), scope);
}
private String getValue(final String key, final Scope scope) {
final StateMap stateMap = getState(scope);
return stateMap.get(key);
}
//
// assertion methods to make unit testing easier
//
/**
* Ensures that the state with the given key and scope is set to the given value, or else the test will fail
*
* @param key the state key
* @param value the expected value
* @param scope the scope
*/
public void assertStateEquals(final String key, final String value, final Scope scope) {
Assert.assertEquals(value, getValue(key, scope));
}
/**
* Ensures that the state is equal to the given values
*
* @param stateValues the values expected
* @param scope the scope to compare the stateValues against
*/
public void assertStateEquals(final Map<String, String> stateValues, final Scope scope) {
final StateMap stateMap = getState(scope);
Assert.assertEquals(stateValues, stateMap.toMap());
}
/**
* Ensures that the state is not equal to the given values
*
* @param stateValues the unexpected values
* @param scope the scope to compare the stateValues against
*/
public void assertStateNotEquals(final Map<String, String> stateValues, final Scope scope) {
final StateMap stateMap = getState(scope);
Assert.assertNotSame(stateValues, stateMap.toMap());
}
/**
* Ensures that the state with the given key and scope is not set to the given value, or else the test will fail
*
* @param key the state key
* @param value the unexpected value
* @param scope the scope
*/
public void assertStateNotEquals(final String key, final String value, final Scope scope) {
Assert.assertNotEquals(value, getValue(key, scope));
}
/**
* Ensures that some value is set for the given key and scope, or else the test will fail
*
* @param key the state key
* @param scope the scope
*/
public void assertStateSet(final String key, final Scope scope) {
Assert.assertNotNull("Expected state to be set for key " + key + " and scope " + scope + ", but it was not set", getValue(key, scope));
}
/**
* Ensures that no value is set for the given key and scope, or else the test will fail
*
* @param key the state key
* @param scope the scope
*/
public void assertStateNotSet(final String key, final Scope scope) {
Assert.assertNull("Expected state not to be set for key " + key + " and scope " + scope + ", but it was set", getValue(key, scope));
}
/**
* Ensures that the state was set for the given scope, regardless of what the value was.
*
* @param scope the scope
*/
public void assertStateSet(final Scope scope) {
final StateMap stateMap = (scope == Scope.CLUSTER) ? clusterStateMap : localStateMap;
Assert.assertEquals("Expected state to be set for Scope " + scope + ", but it was not set", -1L, stateMap.getVersion());
}
/**
* Ensures that the state was not set for the given scope
*
* @param scope the scope
*/
public void assertStateNotSet(final Scope scope) {
final StateMap stateMap = (scope == Scope.CLUSTER) ? clusterStateMap : localStateMap;
Assert.assertNotSame("Expected state not to be set for Scope " + scope + ", but it was set", -1L, stateMap.getVersion());
}
}

View File

@ -0,0 +1,49 @@
/*
* 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.state;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import org.apache.nifi.components.state.StateMap;
public class MockStateMap implements StateMap {
private final Map<String, String> stateValues;
private final long version;
public MockStateMap(final Map<String, String> stateValues, final long version) {
this.stateValues = stateValues == null ? Collections.<String, String> emptyMap() : new HashMap<>(stateValues);
this.version = version;
}
@Override
public long getVersion() {
return version;
}
@Override
public String get(final String key) {
return stateValues.get(key);
}
@Override
public Map<String, String> toMap() {
return Collections.unmodifiableMap(stateValues);
}
}

View File

@ -16,23 +16,31 @@
*/
package org.apache.nifi.util;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.controller.ControllerServiceInitializationContext;
import org.apache.nifi.controller.ControllerServiceLookup;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.state.MockStateManager;
public class MockControllerServiceInitializationContext extends MockControllerServiceLookup implements ControllerServiceInitializationContext, ControllerServiceLookup {
private final String identifier;
private final ComponentLog logger;
private final StateManager stateManager;
public MockControllerServiceInitializationContext(final ControllerService controllerService, final String identifier) {
this(controllerService, identifier, new MockProcessorLog(identifier, controllerService));
this(controllerService, identifier, new MockStateManager());
}
public MockControllerServiceInitializationContext(final ControllerService controllerService, final String identifier, final ComponentLog logger) {
public MockControllerServiceInitializationContext(final ControllerService controllerService, final String identifier, final StateManager stateManager) {
this(controllerService, identifier, new MockProcessorLog(identifier, controllerService), stateManager);
}
public MockControllerServiceInitializationContext(final ControllerService controllerService, final String identifier, final ComponentLog logger, final StateManager stateManager) {
this.identifier = identifier;
this.logger = logger;
this.stateManager = stateManager;
addControllerService(controllerService, identifier);
}
@ -55,4 +63,9 @@ public class MockControllerServiceInitializationContext extends MockControllerSe
public ComponentLog getLogger() {
return logger;
}
@Override
public StateManager getStateManager() {
return stateManager;
}
}

View File

@ -34,17 +34,20 @@ import org.apache.nifi.components.ConfigurableComponent;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.controller.ControllerServiceLookup;
import org.apache.nifi.processor.Processor;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.SchedulingContext;
import org.apache.nifi.state.MockStateManager;
import org.junit.Assert;
public class MockProcessContext extends MockControllerServiceLookup implements SchedulingContext, ControllerServiceLookup {
private final ConfigurableComponent component;
private final Map<PropertyDescriptor, String> properties = new HashMap<>();
private final StateManager stateManager;
private String annotationData = null;
private boolean yieldCalled = false;
@ -56,17 +59,22 @@ public class MockProcessContext extends MockControllerServiceLookup implements S
private volatile Set<Relationship> connections = new HashSet<>();
private volatile Set<Relationship> unavailableRelationships = new HashSet<>();
public MockProcessContext(final ConfigurableComponent component) {
this(component, new MockStateManager());
}
/**
* Creates a new MockProcessContext for the given Processor
*
* @param component being mocked
*/
public MockProcessContext(final ConfigurableComponent component) {
public MockProcessContext(final ConfigurableComponent component, final StateManager stateManager) {
this.component = Objects.requireNonNull(component);
this.stateManager = stateManager;
}
public MockProcessContext(final ControllerService component, final MockProcessContext context) {
this(component);
public MockProcessContext(final ControllerService component, final MockProcessContext context, final StateManager stateManager) {
this(component, stateManager);
try {
annotationData = context.getControllerServiceAnnotationData(component);
@ -121,7 +129,7 @@ public class MockProcessContext extends MockControllerServiceLookup implements S
requireNonNull(value, "Cannot set property to null value; if the intent is to remove the property, call removeProperty instead");
final PropertyDescriptor fullyPopulatedDescriptor = component.getPropertyDescriptor(descriptor.getName());
final ValidationResult result = fullyPopulatedDescriptor.validate(value, new MockValidationContext(this));
final ValidationResult result = fullyPopulatedDescriptor.validate(value, new MockValidationContext(this, stateManager));
String oldValue = properties.put(fullyPopulatedDescriptor, value);
if (oldValue == null) {
oldValue = fullyPopulatedDescriptor.getDefaultValue();
@ -204,7 +212,7 @@ public class MockProcessContext extends MockControllerServiceLookup implements S
* non-null
*/
public Collection<ValidationResult> validate() {
return component.validate(new MockValidationContext(this));
return component.validate(new MockValidationContext(this, stateManager));
}
public boolean isValid() {
@ -342,4 +350,9 @@ public class MockProcessContext extends MockControllerServiceLookup implements S
final List<Range> elRanges = Query.extractExpressionRanges(getProperty(property).getValue());
return (elRanges != null && !elRanges.isEmpty());
}
@Override
public StateManager getStateManager() {
return stateManager;
}
}

View File

@ -24,6 +24,7 @@ import java.util.Map;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.controller.ControllerServiceLookup;
import org.apache.nifi.reporting.Bulletin;
@ -37,11 +38,13 @@ public class MockReportingContext extends MockControllerServiceLookup implements
private final Map<String, ControllerServiceConfiguration> controllerServices;
private final MockEventAccess eventAccess = new MockEventAccess();
private final Map<PropertyDescriptor, String> properties = new HashMap<>();
private final StateManager stateManager;
private final Map<String, List<Bulletin>> componentBulletinsCreated = new HashMap<>();
public MockReportingContext(final Map<String, ControllerService> controllerServices) {
public MockReportingContext(final Map<String, ControllerService> controllerServices, final StateManager stateManager) {
this.controllerServices = new HashMap<>();
this.stateManager = stateManager;
for (final Map.Entry<String, ControllerService> entry : controllerServices.entrySet()) {
this.controllerServices.put(entry.getKey(), new ControllerServiceConfiguration(entry.getValue()));
}
@ -112,4 +115,9 @@ public class MockReportingContext extends MockControllerServiceLookup implements
return new ArrayList<>(created);
}
@Override
public StateManager getStateManager() {
return stateManager;
}
}

View File

@ -27,6 +27,7 @@ import org.apache.nifi.attribute.expression.language.StandardExpressionLanguageC
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.controller.ControllerServiceLookup;
import org.apache.nifi.expression.ExpressionLanguageCompiler;
@ -35,9 +36,11 @@ public class MockValidationContext implements ValidationContext, ControllerServi
private final MockProcessContext context;
private final Map<String, Boolean> expressionLanguageSupported;
private final StateManager stateManager;
public MockValidationContext(final MockProcessContext processContext) {
public MockValidationContext(final MockProcessContext processContext, final StateManager stateManager) {
this.context = processContext;
this.stateManager = stateManager;
final Map<PropertyDescriptor, String> properties = processContext.getProperties();
expressionLanguageSupported = new HashMap<>(properties.size());
@ -63,8 +66,8 @@ public class MockValidationContext implements ValidationContext, ControllerServi
@Override
public ValidationContext getControllerServiceValidationContext(final ControllerService controllerService) {
final MockProcessContext serviceProcessContext = new MockProcessContext(controllerService, context);
return new MockValidationContext(serviceProcessContext);
final MockProcessContext serviceProcessContext = new MockProcessContext(controllerService, context, stateManager);
return new MockValidationContext(serviceProcessContext, stateManager);
}
@Override
@ -118,6 +121,7 @@ public class MockValidationContext implements ValidationContext, ControllerServi
return context.isControllerServiceEnabling(serviceIdentifier);
}
@Override
public boolean isExpressionLanguagePresent(final String value) {
if (value == null) {
return false;

View File

@ -69,6 +69,7 @@ import org.apache.nifi.processor.Relationship;
import org.apache.nifi.provenance.ProvenanceEventRecord;
import org.apache.nifi.provenance.ProvenanceReporter;
import org.apache.nifi.reporting.InitializationException;
import org.apache.nifi.state.MockStateManager;
import org.junit.Assert;
public class StandardProcessorTestRunner implements TestRunner {
@ -80,6 +81,7 @@ public class StandardProcessorTestRunner implements TestRunner {
private final SharedSessionState sharedState;
private final AtomicLong idGenerator;
private final boolean triggerSerially;
private final MockStateManager stateManager;
private int numThreads = 1;
private final AtomicInteger invocations = new AtomicInteger(0);
@ -100,7 +102,8 @@ public class StandardProcessorTestRunner implements TestRunner {
this.sharedState = new SharedSessionState(processor, idGenerator);
this.flowFileQueue = sharedState.getFlowFileQueue();
this.sessionFactory = new MockSessionFactory(sharedState, processor);
this.context = new MockProcessContext(processor);
this.stateManager = new MockStateManager();
this.context = new MockProcessContext(processor, stateManager);
detectDeprecatedAnnotations(processor);
@ -575,7 +578,7 @@ public class StandardProcessorTestRunner implements TestRunner {
// }
final ComponentLog logger = new MockProcessorLog(identifier, service);
final MockControllerServiceInitializationContext initContext = new MockControllerServiceInitializationContext(requireNonNull(service), requireNonNull(identifier), logger);
final MockControllerServiceInitializationContext initContext = new MockControllerServiceInitializationContext(requireNonNull(service), requireNonNull(identifier), logger, stateManager);
initContext.addControllerServices(context);
service.initialize(initContext);
@ -595,7 +598,7 @@ public class StandardProcessorTestRunner implements TestRunner {
@Override
public void assertNotValid(final ControllerService service) {
final ValidationContext validationContext = new MockValidationContext(context).getControllerServiceValidationContext(service);
final ValidationContext validationContext = new MockValidationContext(context, stateManager).getControllerServiceValidationContext(service);
final Collection<ValidationResult> results = context.getControllerService(service.getIdentifier()).validate(validationContext);
for (final ValidationResult result : results) {
@ -609,7 +612,7 @@ public class StandardProcessorTestRunner implements TestRunner {
@Override
public void assertValid(final ControllerService service) {
final ValidationContext validationContext = new MockValidationContext(context).getControllerServiceValidationContext(service);
final ValidationContext validationContext = new MockValidationContext(context, stateManager).getControllerServiceValidationContext(service);
final Collection<ValidationResult> results = context.getControllerService(service.getIdentifier()).validate(validationContext);
for (final ValidationResult result : results) {
@ -719,7 +722,7 @@ public class StandardProcessorTestRunner implements TestRunner {
final Map<PropertyDescriptor, String> curProps = configuration.getProperties();
final Map<PropertyDescriptor, String> updatedProps = new HashMap<>(curProps);
final ValidationContext validationContext = new MockValidationContext(context).getControllerServiceValidationContext(service);
final ValidationContext validationContext = new MockValidationContext(context, stateManager).getControllerServiceValidationContext(service);
final ValidationResult validationResult = property.validate(value, validationContext);
updatedProps.put(property, value);
@ -768,4 +771,8 @@ public class StandardProcessorTestRunner implements TestRunner {
sharedState.clearProvenanceEvents();
}
@Override
public MockStateManager getStateManager() {
return stateManager;
}
}

View File

@ -35,6 +35,7 @@ import org.apache.nifi.processor.Relationship;
import org.apache.nifi.provenance.ProvenanceEventRecord;
import org.apache.nifi.provenance.ProvenanceReporter;
import org.apache.nifi.reporting.InitializationException;
import org.apache.nifi.state.MockStateManager;
public interface TestRunner {
@ -824,4 +825,9 @@ public interface TestRunner {
* Clears the Provenance Events that have been emitted by the Processor
*/
void clearProvenanceEvents();
/**
* @return the State Provider that is used to stored and retrieve local state
*/
MockStateManager getStateManager();
}

View File

@ -16,6 +16,7 @@
*/
package org.apache.nifi.documentation.mock;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.controller.ControllerServiceInitializationContext;
import org.apache.nifi.controller.ControllerServiceLookup;
import org.apache.nifi.logging.ComponentLog;
@ -43,4 +44,9 @@ public class MockControllerServiceInitializationContext implements ControllerSer
return new MockProcessorLogger();
}
@Override
public StateManager getStateManager() {
return null;
}
}

View File

@ -22,6 +22,7 @@ import java.util.Set;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.controller.ControllerServiceLookup;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.Relationship;
@ -102,4 +103,9 @@ public class MockProcessContext implements ProcessContext {
public boolean isExpressionLanguagePresent(PropertyDescriptor property) {
return false;
}
@Override
public StateManager getStateManager() {
return null;
}
}

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.protocol;
import java.util.Collection;
import java.util.Collections;
import javax.xml.bind.annotation.XmlRootElement;
/**
* Represents a set of Node Identifiers that identify all nodes that are in a NiFi Cluster
*/
@XmlRootElement(name = "ClusterNodes")
public class ClusterNodes {
private Collection<NodeIdentifier> nodeIds;
public ClusterNodes(final Collection<NodeIdentifier> nodeIds) {
this.nodeIds = nodeIds;
}
public Collection<NodeIdentifier> getNodeIdentifiers() {
return Collections.unmodifiableCollection(nodeIds);
}
}

View File

@ -31,7 +31,7 @@ import org.apache.nifi.cluster.protocol.jaxb.message.ConnectionResponseAdapter;
@XmlJavaTypeAdapter(ConnectionResponseAdapter.class)
public class ConnectionResponse {
private final boolean blockedByFirewall;
private final String rejectionReason;
private final int tryLaterSeconds;
private final NodeIdentifier nodeIdentifier;
private final StandardDataFlow dataFlow;
@ -52,7 +52,7 @@ public class ConnectionResponse {
this.nodeIdentifier = nodeIdentifier;
this.dataFlow = dataFlow;
this.tryLaterSeconds = 0;
this.blockedByFirewall = false;
this.rejectionReason = null;
this.primary = primary;
this.managerRemoteInputPort = managerRemoteInputPort;
this.managerRemoteCommsSecure = managerRemoteCommsSecure;
@ -66,18 +66,18 @@ public class ConnectionResponse {
this.dataFlow = null;
this.nodeIdentifier = null;
this.tryLaterSeconds = tryLaterSeconds;
this.blockedByFirewall = false;
this.rejectionReason = null;
this.primary = false;
this.managerRemoteInputPort = null;
this.managerRemoteCommsSecure = null;
this.instanceId = null;
}
private ConnectionResponse() {
private ConnectionResponse(final String rejectionReason) {
this.dataFlow = null;
this.nodeIdentifier = null;
this.tryLaterSeconds = 0;
this.blockedByFirewall = true;
this.rejectionReason = rejectionReason;
this.primary = false;
this.managerRemoteInputPort = null;
this.managerRemoteCommsSecure = null;
@ -85,7 +85,15 @@ public class ConnectionResponse {
}
public static ConnectionResponse createBlockedByFirewallResponse() {
return new ConnectionResponse();
return new ConnectionResponse("Blocked by Firewall");
}
public static ConnectionResponse createConflictingNodeIdResponse(final String otherNode) {
return new ConnectionResponse("The Node Identifier provided already belongs to node " + otherNode);
}
public static ConnectionResponse createRejectionResponse(final String explanation) {
return new ConnectionResponse(explanation);
}
public boolean isPrimary() {
@ -96,8 +104,8 @@ public class ConnectionResponse {
return tryLaterSeconds > 0;
}
public boolean isBlockedByFirewall() {
return blockedByFirewall;
public String getRejectionReason() {
return rejectionReason;
}
public int getTryLaterSeconds() {
@ -135,5 +143,4 @@ public class ConnectionResponse {
public String getClusterManagerDN() {
return clusterManagerDN;
}
}

View File

@ -61,13 +61,30 @@ public class NodeIdentifier {
*/
private final int socketPort;
/**
* the IP or hostname that external clients should use to communicate with this node via Site-to-Site
*/
private final String siteToSiteAddress;
/**
* the port that external clients should use to communicate with this node via Site-to-Site
*/
private final Integer siteToSitePort;
/**
* whether or not site-to-site communications with this node are secure
*/
private Boolean siteToSiteSecure;
private final String nodeDn;
public NodeIdentifier(final String id, final String apiAddress, final int apiPort, final String socketAddress, final int socketPort) {
this(id, apiAddress, apiPort, socketAddress, socketPort, null);
public NodeIdentifier(final String id, final String apiAddress, final int apiPort, final String socketAddress, final int socketPort,
final String siteToSiteAddress, final Integer siteToSitePort, final boolean siteToSiteSecure) {
this(id, apiAddress, apiPort, socketAddress, socketPort, siteToSiteAddress, siteToSitePort, siteToSiteSecure, null);
}
public NodeIdentifier(final String id, final String apiAddress, final int apiPort, final String socketAddress, final int socketPort, final String dn) {
public NodeIdentifier(final String id, final String apiAddress, final int apiPort, final String socketAddress, final int socketPort,
final String siteToSiteAddress, final Integer siteToSitePort, final boolean siteToSiteSecure, final String dn) {
if (StringUtils.isBlank(id)) {
throw new IllegalArgumentException("Node ID may not be empty or null.");
@ -79,6 +96,9 @@ public class NodeIdentifier {
validatePort(apiPort);
validatePort(socketPort);
if (siteToSitePort != null) {
validatePort(siteToSitePort);
}
this.id = id;
this.apiAddress = apiAddress;
@ -86,6 +106,9 @@ public class NodeIdentifier {
this.socketAddress = socketAddress;
this.socketPort = socketPort;
this.nodeDn = dn;
this.siteToSiteAddress = siteToSiteAddress;
this.siteToSitePort = siteToSitePort;
this.siteToSiteSecure = siteToSiteSecure;
}
public String getId() {
@ -118,6 +141,19 @@ public class NodeIdentifier {
}
}
public String getSiteToSiteAddress() {
return siteToSiteAddress;
}
public Integer getSiteToSitePort() {
return siteToSitePort;
}
public boolean isSiteToSiteSecure() {
return siteToSiteSecure;
}
/**
* Compares the id of two node identifiers for equality.
*
@ -165,6 +201,7 @@ public class NodeIdentifier {
if (this.socketPort != other.socketPort) {
return false;
}
return true;
}

View File

@ -27,7 +27,7 @@ public class AdaptedConnectionResponse {
private StandardDataFlow dataFlow;
private NodeIdentifier nodeIdentifier;
private boolean blockedByFirewall;
private String rejectionReason;
private boolean primary;
private int tryLaterSeconds;
private Integer managerRemoteInputPort;
@ -63,12 +63,12 @@ public class AdaptedConnectionResponse {
this.tryLaterSeconds = tryLaterSeconds;
}
public boolean isBlockedByFirewall() {
return blockedByFirewall;
public String getRejectionReason() {
return rejectionReason;
}
public void setBlockedByFirewall(boolean blockedByFirewall) {
this.blockedByFirewall = blockedByFirewall;
public void setRejectionReason(final String rejectionReason) {
this.rejectionReason = rejectionReason;
}
public boolean isPrimary() {

View File

@ -21,14 +21,13 @@ package org.apache.nifi.cluster.protocol.jaxb.message;
public class AdaptedNodeIdentifier {
private String id;
private String apiAddress;
private int apiPort;
private String socketAddress;
private int socketPort;
private String siteToSiteAddress;
private Integer siteToSitePort;
private boolean siteToSiteSecure;
public AdaptedNodeIdentifier() {
}
@ -73,4 +72,28 @@ public class AdaptedNodeIdentifier {
this.socketPort = socketPort;
}
public String getSiteToSiteAddress() {
return siteToSiteAddress;
}
public void setSiteToSiteAddress(String siteToSiteAddress) {
this.siteToSiteAddress = siteToSiteAddress;
}
public Integer getSiteToSitePort() {
return siteToSitePort;
}
public void setSiteToSitePort(Integer siteToSitePort) {
this.siteToSitePort = siteToSitePort;
}
public boolean isSiteToSiteSecure() {
return siteToSiteSecure;
}
public void setSiteToSiteSecure(boolean siteToSiteSecure) {
this.siteToSiteSecure = siteToSiteSecure;
}
}

View File

@ -30,7 +30,7 @@ public class ConnectionResponseAdapter extends XmlAdapter<AdaptedConnectionRespo
aCr.setDataFlow(cr.getDataFlow());
aCr.setNodeIdentifier(cr.getNodeIdentifier());
aCr.setTryLaterSeconds(cr.getTryLaterSeconds());
aCr.setBlockedByFirewall(cr.isBlockedByFirewall());
aCr.setRejectionReason(cr.getRejectionReason());
aCr.setPrimary(cr.isPrimary());
aCr.setManagerRemoteInputPort(cr.getManagerRemoteInputPort());
aCr.setManagerRemoteCommsSecure(cr.isManagerRemoteCommsSecure());
@ -43,8 +43,8 @@ public class ConnectionResponseAdapter extends XmlAdapter<AdaptedConnectionRespo
public ConnectionResponse unmarshal(final AdaptedConnectionResponse aCr) {
if (aCr.shouldTryLater()) {
return new ConnectionResponse(aCr.getTryLaterSeconds());
} else if (aCr.isBlockedByFirewall()) {
return ConnectionResponse.createBlockedByFirewallResponse();
} else if (aCr.getRejectionReason() != null) {
return ConnectionResponse.createRejectionResponse(aCr.getRejectionReason());
} else {
return new ConnectionResponse(aCr.getNodeIdentifier(), aCr.getDataFlow(), aCr.isPrimary(),
aCr.getManagerRemoteInputPort(), aCr.isManagerRemoteCommsSecure(), aCr.getInstanceId());

View File

@ -34,6 +34,9 @@ public class NodeIdentifierAdapter extends XmlAdapter<AdaptedNodeIdentifier, Nod
aNi.setApiPort(ni.getApiPort());
aNi.setSocketAddress(ni.getSocketAddress());
aNi.setSocketPort(ni.getSocketPort());
aNi.setSiteToSiteAddress(ni.getSiteToSiteAddress());
aNi.setSiteToSitePort(ni.getSiteToSitePort());
aNi.setSiteToSiteSecure(ni.isSiteToSiteSecure());
return aNi;
}
}
@ -43,7 +46,8 @@ public class NodeIdentifierAdapter extends XmlAdapter<AdaptedNodeIdentifier, Nod
if (aNi == null) {
return null;
} else {
return new NodeIdentifier(aNi.getId(), aNi.getApiAddress(), aNi.getApiPort(), aNi.getSocketAddress(), aNi.getSocketPort());
return new NodeIdentifier(aNi.getId(), aNi.getApiAddress(), aNi.getApiPort(), aNi.getSocketAddress(), aNi.getSocketPort(),
aNi.getSiteToSiteAddress(), aNi.getSiteToSitePort(), aNi.isSiteToSiteSecure());
}
}

View File

@ -87,7 +87,7 @@ public class ClusterManagerProtocolSenderImplTest {
when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(new FlowResponseMessage());
FlowRequestMessage request = new FlowRequestMessage();
request.setNodeId(new NodeIdentifier("id", "api-address", 1, "localhost", port));
request.setNodeId(new NodeIdentifier("id", "api-address", 1, "localhost", port, "localhost", 3821, false));
FlowResponseMessage response = sender.requestFlow(request);
assertNotNull(response);
}
@ -98,7 +98,7 @@ public class ClusterManagerProtocolSenderImplTest {
when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE);
when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(new PingMessage());
FlowRequestMessage request = new FlowRequestMessage();
request.setNodeId(new NodeIdentifier("id", "api-address", 1, "localhost", port));
request.setNodeId(new NodeIdentifier("id", "api-address", 1, "localhost", port, "localhost", 3821, false));
try {
sender.requestFlow(request);
fail("failed to throw exception");
@ -122,7 +122,7 @@ public class ClusterManagerProtocolSenderImplTest {
}
});
FlowRequestMessage request = new FlowRequestMessage();
request.setNodeId(new NodeIdentifier("id", "api-address", 1, "localhost", port));
request.setNodeId(new NodeIdentifier("id", "api-address", 1, "localhost", port, "localhost", 3821, false));
try {
sender.requestFlow(request);
fail("failed to throw exception");

View File

@ -47,7 +47,6 @@ import org.apache.nifi.io.socket.ServerSocketConfiguration;
import org.apache.nifi.io.socket.SocketConfiguration;
import org.apache.nifi.io.socket.multicast.DiscoverableService;
import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl;
import org.junit.After;
import org.junit.Before;
import org.junit.Ignore;
@ -80,7 +79,7 @@ public class NodeProtocolSenderImplTest {
mockServiceLocator = mock(ClusterServiceLocator.class);
mockHandler = mock(ProtocolHandler.class);
nodeIdentifier = new NodeIdentifier("1", "localhost", 1234, "localhost", 5678);
nodeIdentifier = new NodeIdentifier("1", "localhost", 1234, "localhost", 5678, "localhost", 3821, false);
ProtocolContext protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT);
@ -178,7 +177,7 @@ public class NodeProtocolSenderImplTest {
when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(null);
HeartbeatMessage hb = new HeartbeatMessage();
hb.setHeartbeat(new Heartbeat(new NodeIdentifier("id", "localhost", 3, "localhost", 4), false, false, new byte[]{1, 2, 3}));
hb.setHeartbeat(new Heartbeat(new NodeIdentifier("id", "localhost", 3, "localhost", 4, "localhost", 3821, false), false, false, new byte[] {1, 2, 3}));
sender.heartbeat(hb);
}
@ -190,7 +189,7 @@ public class NodeProtocolSenderImplTest {
when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(null);
ControllerStartupFailureMessage msg = new ControllerStartupFailureMessage();
msg.setNodeId(new NodeIdentifier("some-id", "some-addr", 1, "some-addr", 1));
msg.setNodeId(new NodeIdentifier("some-id", "some-addr", 1, "some-addr", 1, "localhost", 3821, false));
msg.setExceptionMessage("some exception");
sender.notifyControllerStartupFailure(msg);
}

View File

@ -17,6 +17,7 @@
package org.apache.nifi.cluster.event;
import java.util.Date;
import org.apache.commons.lang3.StringUtils;
/**

View File

@ -25,6 +25,7 @@ import java.util.List;
import java.util.Map;
import java.util.PriorityQueue;
import java.util.Queue;
import org.apache.nifi.cluster.event.Event;
import org.apache.nifi.cluster.event.EventManager;

View File

@ -24,10 +24,11 @@ import java.net.InetAddress;
import java.net.UnknownHostException;
import java.util.ArrayList;
import java.util.Collection;
import org.apache.commons.net.util.SubnetUtils;
import org.apache.nifi.cluster.firewall.ClusterNodeFirewall;
import org.apache.nifi.util.file.FileUtils;
import org.apache.nifi.logging.NiFiLog;
import org.apache.nifi.util.file.FileUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

View File

@ -16,19 +16,19 @@
*/
package org.apache.nifi.cluster.manager;
import org.apache.nifi.cluster.manager.exception.DisconnectedNodeMutableRequestException;
import org.apache.nifi.cluster.manager.exception.UriConstructionException;
import org.apache.nifi.cluster.manager.exception.ConnectingNodeMutableRequestException;
import org.apache.nifi.cluster.manager.exception.NoConnectedNodesException;
import org.apache.nifi.cluster.manager.exception.NoResponseFromNodesException;
import org.apache.nifi.cluster.manager.exception.SafeModeMutableRequestException;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import java.net.URI;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.nifi.cluster.manager.exception.ConnectingNodeMutableRequestException;
import org.apache.nifi.cluster.manager.exception.DisconnectedNodeMutableRequestException;
import org.apache.nifi.cluster.manager.exception.NoConnectedNodesException;
import org.apache.nifi.cluster.manager.exception.NoResponseFromNodesException;
import org.apache.nifi.cluster.manager.exception.SafeModeMutableRequestException;
import org.apache.nifi.cluster.manager.exception.UriConstructionException;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
/**
* Extends the ClusterManager interface to define how requests issued to the cluster manager are federated to the nodes. Specifically, the HTTP protocol is used for communicating requests to the
* cluster manager and to the nodes.

View File

@ -16,11 +16,12 @@
*/
package org.apache.nifi.cluster.manager;
import org.apache.nifi.cluster.manager.exception.UriConstructionException;
import java.net.URI;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.nifi.cluster.manager.exception.UriConstructionException;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
/**

View File

@ -19,6 +19,7 @@ package org.apache.nifi.cluster.manager;
import java.net.URI;
import java.util.Map;
import java.util.Set;
import org.apache.nifi.cluster.node.Node.Status;
/**

View File

@ -16,7 +16,6 @@
*/
package org.apache.nifi.cluster.manager;
import com.sun.jersey.api.client.ClientResponse;
import java.io.BufferedInputStream;
import java.io.IOException;
import java.io.OutputStream;
@ -31,14 +30,16 @@ import javax.ws.rs.core.Response;
import javax.ws.rs.core.Response.ResponseBuilder;
import javax.ws.rs.core.Response.Status;
import javax.ws.rs.core.StreamingOutput;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.web.api.entity.Entity;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.web.api.entity.Entity;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.sun.jersey.api.client.ClientResponse;
/**
* Encapsulates a node's response in regards to receiving a external API request.
*

View File

@ -0,0 +1,46 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.cluster.manager.exception;
public class ConflictingNodeIdException extends Exception {
private static final long serialVersionUID = 1L;
private final String nodeId;
private final String conflictingNodeAddress;
private final int conflictingNodePort;
public ConflictingNodeIdException(final String nodeId, final String conflictingNodeAddress, final int conflictingNodePort) {
super("Node Identifier " + nodeId + " conflicts with existing node " + conflictingNodeAddress + ":" + conflictingNodePort);
this.nodeId = nodeId;
this.conflictingNodeAddress = conflictingNodeAddress;
this.conflictingNodePort = conflictingNodePort;
}
public String getNodeId() {
return nodeId;
}
public String getConflictingNodeAddress() {
return conflictingNodeAddress;
}
public int getConflictingNodePort() {
return conflictingNodePort;
}
}

View File

@ -16,6 +16,10 @@
*/
package org.apache.nifi.cluster.manager.impl;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.nifi.action.Action;
import org.apache.nifi.admin.service.AuditService;
import org.apache.nifi.controller.status.ProcessGroupStatus;
@ -30,10 +34,6 @@ import org.apache.nifi.provenance.search.QuerySubmission;
import org.apache.nifi.provenance.search.SearchableField;
import org.apache.nifi.reporting.EventAccess;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
public class ClusteredEventAccess implements EventAccess {
private final WebClusterManager clusterManager;

View File

@ -25,6 +25,7 @@ import org.apache.nifi.attribute.expression.language.Query;
import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.controller.ControllerServiceLookup;
import org.apache.nifi.controller.service.ControllerServiceProvider;
@ -47,13 +48,15 @@ public class ClusteredReportingContext implements ReportingContext {
private final ControllerServiceProvider serviceProvider;
private final Map<PropertyDescriptor, String> properties;
private final Map<PropertyDescriptor, PreparedQuery> preparedQueries;
private final StateManager stateManager;
public ClusteredReportingContext(final EventAccess eventAccess, final BulletinRepository bulletinRepository,
final Map<PropertyDescriptor, String> properties, final ControllerServiceProvider serviceProvider) {
public ClusteredReportingContext(final EventAccess eventAccess, final BulletinRepository bulletinRepository, final Map<PropertyDescriptor, String> properties,
final ControllerServiceProvider serviceProvider, final StateManager stateManager) {
this.eventAccess = eventAccess;
this.bulletinRepository = bulletinRepository;
this.properties = Collections.unmodifiableMap(properties);
this.serviceProvider = serviceProvider;
this.stateManager = stateManager;
preparedQueries = new HashMap<>();
for (final Map.Entry<PropertyDescriptor, String> entry : properties.entrySet()) {
@ -206,4 +209,9 @@ public class ClusteredReportingContext implements ReportingContext {
return null;
}
@Override
public StateManager getStateManager() {
return stateManager;
}
}

View File

@ -16,14 +16,6 @@
*/
package org.apache.nifi.cluster.manager.impl;
import com.sun.jersey.api.client.Client;
import com.sun.jersey.api.client.ClientResponse;
import com.sun.jersey.api.client.UniformInterfaceException;
import com.sun.jersey.api.client.WebResource;
import com.sun.jersey.api.client.config.ClientConfig;
import com.sun.jersey.api.client.filter.GZIPContentEncodingFilter;
import com.sun.jersey.core.util.MultivaluedMapImpl;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.ArrayList;
@ -47,17 +39,24 @@ import javax.ws.rs.HttpMethod;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.MultivaluedMap;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.cluster.manager.HttpRequestReplicator;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.manager.exception.UriConstructionException;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.logging.NiFiLog;
import org.apache.nifi.util.FormatUtils;
import org.apache.commons.lang3.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.sun.jersey.api.client.Client;
import com.sun.jersey.api.client.ClientResponse;
import com.sun.jersey.api.client.UniformInterfaceException;
import com.sun.jersey.api.client.WebResource;
import com.sun.jersey.api.client.config.ClientConfig;
import com.sun.jersey.api.client.filter.GZIPContentEncodingFilter;
import com.sun.jersey.core.util.MultivaluedMapImpl;
/**
* An implementation of the <code>HttpRequestReplicator</code> interface. This implementation parallelizes the node HTTP requests using the given <code>ExecutorService</code> instance. Individual
* requests may have connection and read timeouts set, which may be set during instance construction. Otherwise, the default is not to timeout.

View File

@ -20,6 +20,7 @@ import java.net.URI;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import org.apache.nifi.cluster.manager.HttpResponseMapper;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.node.Node;

View File

@ -87,6 +87,7 @@ import org.apache.nifi.cluster.manager.HttpClusterManager;
import org.apache.nifi.cluster.manager.HttpRequestReplicator;
import org.apache.nifi.cluster.manager.HttpResponseMapper;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.manager.exception.ConflictingNodeIdException;
import org.apache.nifi.cluster.manager.exception.ConnectingNodeMutableRequestException;
import org.apache.nifi.cluster.manager.exception.DisconnectedNodeMutableRequestException;
import org.apache.nifi.cluster.manager.exception.IllegalClusterStateException;
@ -126,6 +127,7 @@ import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType;
import org.apache.nifi.cluster.protocol.message.ReconnectionFailureMessage;
import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.state.StateManagerProvider;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.controller.FlowFileSummaries;
import org.apache.nifi.controller.Heartbeater;
@ -151,6 +153,7 @@ import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.controller.service.ControllerServiceProvider;
import org.apache.nifi.controller.service.ControllerServiceState;
import org.apache.nifi.controller.service.StandardControllerServiceProvider;
import org.apache.nifi.controller.state.manager.StandardStateManagerProvider;
import org.apache.nifi.controller.status.ProcessGroupStatus;
import org.apache.nifi.controller.status.RemoteProcessGroupStatus;
import org.apache.nifi.controller.status.history.ComponentStatusRepository;
@ -371,8 +374,10 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C
private final FlowEngine reportingTaskEngine;
private final Map<NodeIdentifier, ComponentStatusRepository> componentMetricsRepositoryMap = new HashMap<>();
private final StandardProcessScheduler processScheduler;
private final StateManagerProvider stateManagerProvider;
private final long componentStatusSnapshotMillis;
public WebClusterManager(final HttpRequestReplicator httpRequestReplicator, final HttpResponseMapper httpResponseMapper,
final DataFlowManagementService dataFlowManagementService, final ClusterManagerProtocolSenderListener senderListener,
final NiFiProperties properties, final StringEncryptor encryptor, final OptimisticLockingManager optimisticLockingManager) {
@ -468,11 +473,17 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C
reportingTaskEngine = new FlowEngine(8, "Reporting Task Thread");
try {
this.stateManagerProvider = StandardStateManagerProvider.create(properties);
} catch (final IOException e) {
throw new RuntimeException(e);
}
processScheduler = new StandardProcessScheduler(new Heartbeater() {
@Override
public void heartbeat() {
}
}, this, encryptor);
}, this, encryptor, stateManagerProvider);
// When we construct the scheduling agents, we can pass null for a lot of the arguments because we are only
// going to be scheduling Reporting Tasks. Otherwise, it would not be okay.
@ -481,13 +492,12 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C
processScheduler.setMaxThreadCount(SchedulingStrategy.TIMER_DRIVEN, 10);
processScheduler.setMaxThreadCount(SchedulingStrategy.CRON_DRIVEN, 10);
controllerServiceProvider = new StandardControllerServiceProvider(processScheduler, bulletinRepository);
controllerServiceProvider = new StandardControllerServiceProvider(processScheduler, bulletinRepository, stateManagerProvider);
}
public void start() throws IOException {
writeLock.lock();
try {
if (isRunning()) {
throw new IllegalStateException("Instance is already started.");
}
@ -712,7 +722,14 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C
try {
// resolve the proposed node identifier to a valid node identifier
final NodeIdentifier resolvedNodeIdentifier = resolveProposedNodeIdentifier(request.getProposedNodeIdentifier());
final NodeIdentifier resolvedNodeIdentifier;
try {
resolvedNodeIdentifier = resolveProposedNodeIdentifier(request.getProposedNodeIdentifier());
} catch (final ConflictingNodeIdException e) {
logger.info("Rejecting node {} from connecting to cluster because it provided a Node ID of {} but that Node ID already belongs to {}:{}",
request.getProposedNodeIdentifier().getSocketAddress(), request.getProposedNodeIdentifier().getId(), e.getConflictingNodeAddress(), e.getConflictingNodePort());
return ConnectionResponse.createConflictingNodeIdResponse(e.getConflictingNodeAddress() + ":" + e.getConflictingNodePort());
}
if (isBlockedByFirewall(resolvedNodeIdentifier.getSocketAddress())) {
// if the socket address is not listed in the firewall, then return a null response
@ -1029,7 +1046,7 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C
for (final Map.Entry<PropertyDescriptor, String> entry : resolvedProps.entrySet()) {
if (entry.getValue() != null) {
reportingTaskNode.setProperty(entry.getKey().getName(), entry.getValue());
reportingTaskNode.setProperty(entry.getKey().getName(), entry.getValue(), false);
}
}
@ -1096,7 +1113,8 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C
final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(this);
final ReportingTaskNode taskNode = new ClusteredReportingTaskNode(task, id, processScheduler,
new ClusteredEventAccess(this, auditService), bulletinRepository, controllerServiceProvider, validationContextFactory);
new ClusteredEventAccess(this, auditService), bulletinRepository, controllerServiceProvider,
validationContextFactory, stateManagerProvider.getStateManager(id));
taskNode.setName(task.getClass().getSimpleName());
reportingTasks.put(id, taskNode);
@ -1354,8 +1372,9 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C
}
private NodeIdentifier addRequestorDn(final NodeIdentifier nodeId, final String dn) {
return new NodeIdentifier(nodeId.getId(), nodeId.getApiAddress(),
nodeId.getApiPort(), nodeId.getSocketAddress(), nodeId.getSocketPort(), dn);
return new NodeIdentifier(nodeId.getId(), nodeId.getApiAddress(), nodeId.getApiPort(),
nodeId.getSocketAddress(), nodeId.getSocketPort(),
nodeId.getSiteToSiteAddress(), nodeId.getSiteToSitePort(), nodeId.isSiteToSiteSecure(), dn);
}
private ConnectionResponseMessage handleConnectionRequest(final ConnectionRequestMessage requestMessage) {
@ -1848,6 +1867,7 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C
}
}
private ComponentStatusRepository createComponentStatusRepository() {
final String implementationClassName = properties.getProperty(NiFiProperties.COMPONENT_STATUS_REPOSITORY_IMPLEMENTATION, DEFAULT_COMPONENT_STATUS_REPO_IMPLEMENTATION);
if (implementationClassName == null) {
@ -3644,7 +3664,7 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C
*
* @return the node identifier that should be used
*/
private NodeIdentifier resolveProposedNodeIdentifier(final NodeIdentifier proposedNodeId) {
private NodeIdentifier resolveProposedNodeIdentifier(final NodeIdentifier proposedNodeId) throws ConflictingNodeIdException {
readLock.lock();
try {
for (final Node node : nodes) {
@ -3660,32 +3680,32 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C
// we know about this node and it has the same ID, so the proposal is fine
return proposedNodeId;
} else if (sameId && !sameServiceCoordinates) {
// proposed ID conflicts with existing node ID, so assign a new ID
final NodeIdentifier resolvedIdentifier = new NodeIdentifier(
UUID.randomUUID().toString(),
proposedNodeId.getApiAddress(),
proposedNodeId.getApiPort(),
proposedNodeId.getSocketAddress(),
proposedNodeId.getSocketPort());
logger.info(String.format("Using Node Identifier %s because proposed node identifier %s conflicts existing node identifiers",
resolvedIdentifier, proposedNodeId));
return resolvedIdentifier;
throw new ConflictingNodeIdException(nodeId.getId(), node.getNodeId().getApiAddress(), node.getNodeId().getApiPort());
} else if (!sameId && sameServiceCoordinates) {
// we know about this node, so we'll use the existing ID
logger.debug(String.format("Using Node Identifier %s because proposed node identifier %s matches the service coordinates",
nodeId, proposedNodeId));
return nodeId;
logger.debug(String.format("Using Node Identifier %s because proposed node identifier %s matches the service coordinates", nodeId, proposedNodeId));
// return a new Node Identifier that uses the existing Node UUID, Node Index, and ZooKeeper Port from the existing Node (because these are the
// elements that are assigned by the NCM), but use the other parameters from the proposed identifier, since these elements are determined by
// the node rather than the NCM.
return new NodeIdentifier(nodeId.getId(),
proposedNodeId.getApiAddress(), proposedNodeId.getApiPort(),
proposedNodeId.getSocketAddress(), proposedNodeId.getSocketPort(),
proposedNodeId.getSiteToSiteAddress(), proposedNodeId.getSiteToSitePort(), proposedNodeId.isSiteToSiteSecure());
}
}
// proposal does not conflict with existing nodes
return proposedNodeId;
// proposal does not conflict with existing nodes - this is a new node. Assign a new Node Index to it
return new NodeIdentifier(proposedNodeId.getId(), proposedNodeId.getApiAddress(), proposedNodeId.getApiPort(),
proposedNodeId.getSocketAddress(), proposedNodeId.getSocketPort(),
proposedNodeId.getSiteToSiteAddress(), proposedNodeId.getSiteToSitePort(), proposedNodeId.isSiteToSiteSecure());
} finally {
readLock.unlock("resolveProposedNodeIdentifier");
}
}
private boolean isHeartbeatMonitorRunning() {
readLock.lock();
try {
@ -3879,13 +3899,13 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C
continue;
}
final Integer siteToSitePort = heartbeat.getSiteToSitePort();
final Integer siteToSitePort = id.getSiteToSitePort();
if (siteToSitePort == null) {
continue;
}
final int flowFileCount = (int) heartbeat.getTotalFlowFileCount();
final NodeInformation nodeInfo = new NodeInformation(id.getApiAddress(), siteToSitePort, id.getApiPort(),
heartbeat.isSiteToSiteSecure(), flowFileCount);
final NodeInformation nodeInfo = new NodeInformation(id.getSiteToSiteAddress(), siteToSitePort, id.getApiPort(),
id.isSiteToSiteSecure(), flowFileCount);
nodeInfos.add(nodeInfo);
}

View File

@ -22,7 +22,6 @@ import org.apache.nifi.cluster.HeartbeatPayload;
import org.apache.nifi.cluster.protocol.Heartbeat;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.cluster.protocol.ProtocolException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

View File

@ -25,7 +25,6 @@ import org.apache.nifi.io.socket.multicast.DiscoverableService;
import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.util.NiFiProperties;
import org.springframework.beans.BeansException;
import org.springframework.beans.factory.DisposableBean;
import org.springframework.beans.factory.FactoryBean;

View File

@ -17,6 +17,7 @@
package org.apache.nifi.cluster.spring;
import java.io.File;
import org.apache.nifi.cluster.firewall.impl.FileBasedClusterNodeFirewall;
import org.apache.nifi.util.NiFiProperties;
import org.springframework.beans.factory.FactoryBean;

View File

@ -17,6 +17,7 @@
package org.apache.nifi.controller.reporting;
import org.apache.nifi.cluster.manager.impl.ClusteredReportingContext;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.controller.ProcessScheduler;
import org.apache.nifi.controller.ValidationContextFactory;
import org.apache.nifi.controller.service.ControllerServiceProvider;
@ -30,20 +31,22 @@ public class ClusteredReportingTaskNode extends AbstractReportingTaskNode {
private final EventAccess eventAccess;
private final BulletinRepository bulletinRepository;
private final ControllerServiceProvider serviceProvider;
private final StateManager stateManager;
public ClusteredReportingTaskNode(final ReportingTask reportingTask, final String id, final ProcessScheduler scheduler,
final EventAccess eventAccess, final BulletinRepository bulletinRepository, final ControllerServiceProvider serviceProvider,
final ValidationContextFactory validationContextFactory) {
final ValidationContextFactory validationContextFactory, final StateManager stateManager) {
super(reportingTask, id, serviceProvider, scheduler, validationContextFactory);
this.eventAccess = eventAccess;
this.bulletinRepository = bulletinRepository;
this.serviceProvider = serviceProvider;
this.stateManager = stateManager;
}
@Override
public ReportingContext getReportingContext() {
return new ClusteredReportingContext(eventAccess, bulletinRepository, getProperties(), serviceProvider);
return new ClusteredReportingContext(eventAccess, bulletinRepository, getProperties(), serviceProvider, stateManager);
}
}

View File

@ -16,15 +16,17 @@
*/
package org.apache.nifi.cluster.event.impl;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import org.apache.nifi.cluster.event.Event;
import org.apache.nifi.cluster.event.Event.Category;
import org.apache.nifi.cluster.event.EventManager;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import org.apache.nifi.cluster.event.Event;
import org.apache.nifi.cluster.event.Event.Category;
import org.apache.nifi.cluster.event.EventManager;
import org.junit.Test;
/**

View File

@ -16,13 +16,15 @@
*/
package org.apache.nifi.cluster.firewall.impl;
import java.io.File;
import java.net.InetAddress;
import java.net.UnknownHostException;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assume.assumeTrue;
import java.io.File;
import java.net.InetAddress;
import java.net.UnknownHostException;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Rule;

View File

@ -30,6 +30,7 @@ import javax.xml.parsers.DocumentBuilder;
import javax.xml.parsers.DocumentBuilderFactory;
import javax.xml.parsers.ParserConfigurationException;
import org.apache.commons.io.FileUtils;
import org.apache.nifi.cluster.flow.DataFlowDao;
import org.apache.nifi.cluster.flow.PersistedFlowState;
import org.apache.nifi.cluster.protocol.ClusterManagerProtocolSender;
@ -46,8 +47,6 @@ import org.apache.nifi.cluster.protocol.message.FlowResponseMessage;
import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
import org.apache.nifi.io.socket.ServerSocketConfiguration;
import org.apache.nifi.io.socket.SocketConfiguration;
import org.apache.commons.io.FileUtils;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@ -148,7 +147,7 @@ public class DataFlowManagementServiceImplTest {
byte[] flowBytes = flowStr.getBytes();
listener.addHandler(new FlowRequestProtocolHandler(new StandardDataFlow(flowStr.getBytes("UTF-8"), new byte[0], new byte[0])));
NodeIdentifier nodeId = new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort);
NodeIdentifier nodeId = new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort, "localhost", 1234, false);
service.setNodeIds(new HashSet<>(Arrays.asList(nodeId)));
service.setPersistedFlowState(PersistedFlowState.STALE);
@ -168,8 +167,8 @@ public class DataFlowManagementServiceImplTest {
String flowStr = "<rootGroup />";
listener.addHandler(new FlowRequestProtocolHandler(new StandardDataFlow(flowStr.getBytes("UTF-8"), new byte[0], new byte[0])));
NodeIdentifier nodeId1 = new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort);
NodeIdentifier nodeId2 = new NodeIdentifier("2", "localhost", apiDummyPort, "localhost", socketPort);
NodeIdentifier nodeId1 = new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort, "localhost", 1234, false);
NodeIdentifier nodeId2 = new NodeIdentifier("2", "localhost", apiDummyPort, "localhost", socketPort, "localhost", 1234, false);
service.setNodeIds(new HashSet<>(Arrays.asList(nodeId1, nodeId2)));
service.setPersistedFlowState(PersistedFlowState.STALE);
@ -196,8 +195,8 @@ public class DataFlowManagementServiceImplTest {
byte[] flowBytes = flowStr.getBytes();
listener.addHandler(new FlowRequestProtocolHandler(new StandardDataFlow(flowStr.getBytes("UTF-8"), new byte[0], new byte[0])));
NodeIdentifier nodeId1 = new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort + 1);
NodeIdentifier nodeId2 = new NodeIdentifier("2", "localhost", apiDummyPort, "localhost", socketPort);
NodeIdentifier nodeId1 = new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort + 1, "localhost", 1234, false);
NodeIdentifier nodeId2 = new NodeIdentifier("2", "localhost", apiDummyPort, "localhost", socketPort, "localhost", 1234, false);
service.setNodeIds(new HashSet<>(Arrays.asList(nodeId1, nodeId2)));
service.setPersistedFlowState(PersistedFlowState.STALE);
@ -217,8 +216,8 @@ public class DataFlowManagementServiceImplTest {
byte[] flowBytes = flowStr.getBytes();
listener.addHandler(new FlowRequestProtocolHandler(new StandardDataFlow(flowStr.getBytes("UTF-8"), new byte[0], new byte[0])));
NodeIdentifier nodeId1 = new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort + 1);
NodeIdentifier nodeId2 = new NodeIdentifier("2", "localhost", apiDummyPort, "localhost", socketPort);
NodeIdentifier nodeId1 = new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort + 1, "localhost", 1234, false);
NodeIdentifier nodeId2 = new NodeIdentifier("2", "localhost", apiDummyPort, "localhost", socketPort, "localhost", 1234, false);
for (int i = 0; i < 1000; i++) {
service.setNodeIds(new HashSet<>(Arrays.asList(nodeId1, nodeId2)));
@ -239,8 +238,8 @@ public class DataFlowManagementServiceImplTest {
String flowStr = "<rootGroup />";
listener.addHandler(new FlowRequestProtocolHandler(new StandardDataFlow(flowStr.getBytes("UTF-8"), new byte[0], new byte[0])));
NodeIdentifier nodeId1 = new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort + 1);
NodeIdentifier nodeId2 = new NodeIdentifier("2", "localhost", apiDummyPort, "localhost", socketPort);
NodeIdentifier nodeId1 = new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort + 1, "localhost", 1234, false);
NodeIdentifier nodeId2 = new NodeIdentifier("2", "localhost", apiDummyPort, "localhost", socketPort, "localhost", 1234, false);
service.setRetrievalDelay("5 sec");
for (int i = 0; i < 1000; i++) {
@ -263,9 +262,9 @@ public class DataFlowManagementServiceImplTest {
listener.addHandler(new FlowRequestProtocolHandler(new StandardDataFlow(flowStr.getBytes("UTF-8"), new byte[0], new byte[0])));
Set<NodeIdentifier> nodeIds = new HashSet<>();
for (int i = 0; i < 1000; i++) {
nodeIds.add(new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort + 1));
nodeIds.add(new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort + 1, "localhost", 1234, false));
}
nodeIds.add(new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort));
nodeIds.add(new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort, "localhost", 1234, false));
long lastRetrievalTime = service.getLastRetrievalTime();
@ -291,7 +290,7 @@ public class DataFlowManagementServiceImplTest {
String flowStr = "<rootGroup />";
byte[] flowBytes = flowStr.getBytes();
listener.addHandler(new FlowRequestProtocolHandler(new StandardDataFlow(flowStr.getBytes("UTF-8"), new byte[0], new byte[0])));
NodeIdentifier nodeId = new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort);
NodeIdentifier nodeId = new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort, "localhost", 1234, false);
service.setNodeIds(new HashSet<>(Arrays.asList(nodeId)));
service.setPersistedFlowState(PersistedFlowState.STALE);

View File

@ -16,36 +16,40 @@
*/
package org.apache.nifi.cluster.manager.impl;
import javax.ws.rs.core.Response;
import javax.xml.bind.annotation.XmlRootElement;
import javax.ws.rs.HttpMethod;
import javax.ws.rs.core.MultivaluedMap;
import java.net.URISyntaxException;
import java.util.Arrays;
import java.util.Iterator;
import javax.ws.rs.core.StreamingOutput;
import org.apache.nifi.cluster.manager.testutils.HttpResponse;
import org.apache.nifi.cluster.manager.testutils.HttpServer;
import com.sun.jersey.api.client.Client;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.net.URI;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import javax.ws.rs.core.Response.Status;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.apache.nifi.cluster.manager.testutils.HttpResponseAction;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import javax.ws.rs.HttpMethod;
import javax.ws.rs.core.MultivaluedMap;
import javax.ws.rs.core.Response;
import javax.ws.rs.core.Response.Status;
import javax.ws.rs.core.StreamingOutput;
import javax.xml.bind.annotation.XmlRootElement;
import org.apache.nifi.cluster.manager.NodeResponse;
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.junit.After;
import org.junit.Before;
import org.junit.Test;
import com.sun.jersey.api.client.Client;
/**
*/
public class HttpRequestReplicatorImplTest {
@ -341,7 +345,7 @@ public class HttpRequestReplicatorImplTest {
private Set<NodeIdentifier> createNodes(int num, String host, int apiPort) {
Set<NodeIdentifier> result = new HashSet<>();
for (int i = 0; i < num; i++) {
result.add(new NodeIdentifier(String.valueOf(i), host, apiPort, host, 1));
result.add(new NodeIdentifier(String.valueOf(i), host, apiPort, host, 1, "localhost", 1234, false));
}
return result;
}

View File

@ -16,23 +16,26 @@
*/
package org.apache.nifi.cluster.manager.impl;
import com.sun.jersey.api.client.ClientResponse;
import com.sun.jersey.core.util.MultivaluedMapImpl;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.io.ByteArrayInputStream;
import java.util.Map;
import java.util.HashSet;
import java.util.Set;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.node.Node;
import org.apache.nifi.cluster.node.Node.Status;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import static org.junit.Assert.assertTrue;
import org.junit.Before;
import org.junit.Test;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import com.sun.jersey.api.client.ClientResponse;
import com.sun.jersey.core.util.MultivaluedMapImpl;
/**
*/
@ -119,7 +122,7 @@ public class HttpResponseMapperImplTest {
when(clientResponse.getHeaders()).thenReturn(new MultivaluedMapImpl());
when(clientResponse.getEntityInputStream()).thenReturn(new ByteArrayInputStream(new byte[0]));
NodeIdentifier nodeIdentifier = new NodeIdentifier(nodeId, "localhost", 1, "localhost", 1);
NodeIdentifier nodeIdentifier = new NodeIdentifier(nodeId, "localhost", 1, "localhost", 1, "localhost", 1234, false);
return new NodeResponse(nodeIdentifier, "GET", dummyUri, clientResponse, 1L, "111");
}
}

View File

@ -16,8 +16,6 @@
*/
package org.apache.nifi.cluster.manager.impl;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import static org.junit.Assert.assertEquals;
import java.text.DateFormat;

View File

@ -25,8 +25,10 @@ import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import javax.ws.rs.HttpMethod;
import javax.ws.rs.core.MediaType;
import org.apache.commons.lang3.StringUtils;
/**

View File

@ -19,6 +19,7 @@ package org.apache.nifi.cluster.manager.testutils;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import javax.ws.rs.core.Response.Status;
/**

View File

@ -32,6 +32,7 @@ import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import org.apache.nifi.cluster.manager.testutils.HttpRequest.HttpRequestBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

View File

@ -0,0 +1,60 @@
/*
* 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.components.state;
/**
* <p>
* Interface that provides a mechanism for obtaining the {@link StateManager} for a particular component
* </p>
*/
public interface StateManagerProvider {
/**
* Returns the StateManager for the component with the given ID, or <code>null</code> if no State Manager
* exists for the component with the given ID
*
* @param componentId the id of the component for which the StateManager should be returned
*
* @return the StateManager for the component with the given ID, or <code>null</code> if no State Manager
* exists for the component with the given ID
*/
StateManager getStateManager(String componentId);
/**
* Notifies the State Manager Provider that the component with the given ID has been removed from the NiFi instance
* and will no longer be needed, so the appropriate resource cleanup can take place.
*
* @param componentId the ID of the component that has been removed
*/
void onComponentRemoved(String componentId);
/**
* Shuts down the state managers, cleaning up any resources that they occupy
*/
void shutdown();
/**
* Initializes the Cluster State Provider and enables it for use
*/
void enableClusterProvider();
/**
* Disables the Cluster State Provider and begins using the Local State Provider to persist and retrieve
* state, even when components request a clustered provider
*/
void disableClusterProvider();
}

View File

@ -85,7 +85,7 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone
}
@Override
public void setProperty(final String name, final String value) {
public void setProperty(final String name, final String value, final boolean triggerOnPropertyModified) {
if (null == name || null == value) {
throw new IllegalArgumentException();
}
@ -114,13 +114,15 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone
}
}
if (triggerOnPropertyModified) {
try {
component.onPropertyModified(descriptor, oldValue, value);
} catch (final Throwable t) {
} catch (final Exception e) {
// nothing really to do here...
}
}
}
}
} finally {
lock.unlock();
}
@ -133,11 +135,12 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone
* if was a dynamic property.
*
* @param name the property to remove
* @param triggerOnPropertyModified specifies whether or not the onPropertyModified method should be called
* @return true if removed; false otherwise
* @throws java.lang.IllegalArgumentException if the name is null
*/
@Override
public boolean removeProperty(final String name) {
public boolean removeProperty(final String name, final boolean triggerOnPropertyModified) {
if (null == name) {
throw new IllegalArgumentException();
}
@ -160,7 +163,10 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone
}
}
if (triggerOnPropertyModified) {
component.onPropertyModified(descriptor, value, null);
}
return true;
}
}

View File

@ -34,7 +34,15 @@ public interface ConfiguredComponent {
public void setAnnotationData(String data);
public void setProperty(String name, String value);
/**
* Sets the property with the given name to the given value
*
* @param name the name of the property to update
* @param value the value to update the property to
* @param triggerOnPropertyModified if <code>true</code>, will trigger the #onPropertyModified method of the component
* to be called, otherwise will not
*/
public void setProperty(String name, String value, boolean triggerOnPropertyModified);
/**
* Removes the property and value for the given property name if a
@ -43,10 +51,12 @@ public interface ConfiguredComponent {
* if was a dynamic property.
*
* @param name the property to remove
* @param triggerOnPropertyModified if <code>true</code>, will trigger the #onPropertyModified method of the component
* to be called, otherwise will not
* @return true if removed; false otherwise
* @throws java.lang.IllegalArgumentException if the name is null
*/
public boolean removeProperty(String name);
public boolean removeProperty(String name, boolean triggerOnPropertyModified);
public Map<PropertyDescriptor, String> getProperties();

View File

@ -1,19 +1,16 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<!-- 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. -->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.nifi</groupId>
@ -119,6 +116,27 @@
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-write-ahead-log</artifactId>
</dependency>
<dependency>
<groupId>org.apache.zookeeper</groupId>
<artifactId>zookeeper</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-processor-utils</artifactId>
</dependency>
<dependency>
<groupId>org.apache.curator</groupId>
<artifactId>curator-test</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.testng</groupId>
<artifactId>testng</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-mock</artifactId>

View File

@ -56,8 +56,6 @@ public class HeartbeatPayload {
private long totalFlowFileCount;
private long totalFlowFileBytes;
private SystemDiagnostics systemDiagnostics;
private Integer siteToSitePort;
private boolean siteToSiteSecure;
private long systemStartTime;
@XmlJavaTypeAdapter(CounterAdapter.class)
@ -109,22 +107,6 @@ public class HeartbeatPayload {
this.systemDiagnostics = systemDiagnostics;
}
public boolean isSiteToSiteSecure() {
return siteToSiteSecure;
}
public void setSiteToSiteSecure(final boolean secure) {
this.siteToSiteSecure = secure;
}
public Integer getSiteToSitePort() {
return siteToSitePort;
}
public void setSiteToSitePort(final Integer port) {
this.siteToSitePort = port;
}
public long getSystemStartTime() {
return systemStartTime;
}

View File

@ -16,7 +16,39 @@
*/
package org.apache.nifi.controller;
import com.sun.jersey.api.client.ClientHandlerException;
import static java.util.Objects.requireNonNull;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.text.DateFormat;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Date;
import java.util.HashSet;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import javax.net.ssl.SSLContext;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.action.Action;
import org.apache.nifi.admin.service.AuditService;
@ -37,6 +69,7 @@ import org.apache.nifi.cluster.protocol.UnknownServiceAddressException;
import org.apache.nifi.cluster.protocol.message.HeartbeatMessage;
import org.apache.nifi.cluster.protocol.message.NodeBulletinsMessage;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.state.StateManagerProvider;
import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.connectable.ConnectableType;
import org.apache.nifi.connectable.Connection;
@ -88,6 +121,8 @@ import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.controller.service.ControllerServiceProvider;
import org.apache.nifi.controller.service.StandardConfigurationContext;
import org.apache.nifi.controller.service.StandardControllerServiceProvider;
import org.apache.nifi.controller.state.manager.StandardStateManagerProvider;
import org.apache.nifi.controller.state.server.ZooKeeperStateServer;
import org.apache.nifi.controller.status.ConnectionStatus;
import org.apache.nifi.controller.status.PortStatus;
import org.apache.nifi.controller.status.ProcessGroupStatus;
@ -173,41 +208,11 @@ import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO;
import org.apache.nifi.web.api.dto.TemplateDTO;
import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.net.ssl.SSLContext;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.text.DateFormat;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Date;
import java.util.HashSet;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import static java.util.Objects.requireNonNull;
import com.sun.jersey.api.client.ClientHandlerException;
public class FlowController implements EventAccess, ControllerServiceProvider, ReportingTaskProvider, Heartbeater, QueueProvider {
@ -251,9 +256,12 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
private final AuditService auditService;
private final EventDrivenWorkerQueue eventDrivenWorkerQueue;
private final ComponentStatusRepository componentStatusRepository;
private final StateManagerProvider stateManagerProvider;
private final long systemStartTime = System.currentTimeMillis(); // time at which the node was started
private final ConcurrentMap<String, ReportingTaskNode> reportingTasks = new ConcurrentHashMap<>();
private volatile ZooKeeperStateServer zooKeeperStateServer;
// The Heartbeat Bean is used to provide an Atomic Reference to data that is used in heartbeats that may
// change while the instance is running. We do this because we want to generate heartbeats even if we
// are unable to obtain a read lock on the entire FlowController.
@ -419,13 +427,19 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
throw new RuntimeException("Unable to create Provenance Repository", e);
}
processScheduler = new StandardProcessScheduler(this, this, encryptor);
try {
this.stateManagerProvider = StandardStateManagerProvider.create(properties);
} catch (final IOException e) {
throw new RuntimeException(e);
}
processScheduler = new StandardProcessScheduler(this, this, encryptor, stateManagerProvider);
eventDrivenWorkerQueue = new EventDrivenWorkerQueue(false, false, processScheduler);
controllerServiceProvider = new StandardControllerServiceProvider(processScheduler, bulletinRepository);
controllerServiceProvider = new StandardControllerServiceProvider(processScheduler, bulletinRepository, stateManagerProvider);
final ProcessContextFactory contextFactory = new ProcessContextFactory(contentRepository, flowFileRepository, flowFileEventRepository, counterRepositoryRef.get(), provenanceEventRepository);
processScheduler.setSchedulingAgent(SchedulingStrategy.EVENT_DRIVEN, new EventDrivenSchedulingAgent(
eventDrivenEngineRef.get(), this, eventDrivenWorkerQueue, contextFactory, maxEventDrivenThreads.get(), encryptor));
eventDrivenEngineRef.get(), this, stateManagerProvider, eventDrivenWorkerQueue, contextFactory, maxEventDrivenThreads.get(), encryptor));
final QuartzSchedulingAgent quartzSchedulingAgent = new QuartzSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor);
final TimerDrivenSchedulingAgent timerDrivenAgent = new TimerDrivenSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor);
@ -469,7 +483,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
this.snippetManager = new SnippetManager();
rootGroup = new StandardProcessGroup(UUID.randomUUID().toString(), this, processScheduler, properties, encryptor);
rootGroup = new StandardProcessGroup(UUID.randomUUID().toString(), this, processScheduler, properties, encryptor, this);
rootGroup.setName(DEFAULT_ROOT_GROUP_NAME);
instanceId = UUID.randomUUID().toString();
@ -496,6 +510,17 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
snapshotMillis = FormatUtils.getTimeDuration(NiFiProperties.DEFAULT_COMPONENT_STATUS_SNAPSHOT_FREQUENCY, TimeUnit.MILLISECONDS);
}
// Initialize the Embedded ZooKeeper server, if applicable
if (properties.isStartEmbeddedZooKeeper()) {
try {
zooKeeperStateServer = ZooKeeperStateServer.create(properties);
} catch (final IOException | ConfigException e) {
throw new IllegalStateException("Unable to initailize Flow because NiFi was configured to start an Embedded Zookeeper server but failed to do so", e);
}
} else {
zooKeeperStateServer = null;
}
componentStatusRepository = createComponentStatusRepository();
timerDrivenEngineRef.get().scheduleWithFixedDelay(new Runnable() {
@Override
@ -668,6 +693,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
}
}
private ContentRepository createContentRepository(final NiFiProperties properties) throws InstantiationException, IllegalAccessException, ClassNotFoundException {
final String implementationClassName = properties.getProperty(NiFiProperties.CONTENT_REPOSITORY_IMPLEMENTATION, DEFAULT_CONTENT_REPO_IMPLEMENTATION);
if (implementationClassName == null) {
@ -714,6 +740,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
}
}
/**
* Creates a connection between two Connectable objects.
*
@ -835,7 +862,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
* @throws NullPointerException if the argument is null
*/
public ProcessGroup createProcessGroup(final String id) {
return new StandardProcessGroup(requireNonNull(id).intern(), this, processScheduler, properties, encryptor);
return new StandardProcessGroup(requireNonNull(id).intern(), this, processScheduler, properties, encryptor, this);
}
/**
@ -945,6 +972,10 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
return snippetManager;
}
public StateManagerProvider getStateManagerProvider() {
return stateManagerProvider;
}
/**
* Creates a Port to use as an Input Port for the root Process Group, which is used for Site-to-Site communications
*
@ -1021,6 +1052,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
}
}
/**
* Sets the name for the Root Group, which also changes the name for the controller.
*
@ -1106,6 +1138,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
// Trigger any processors' methods marked with @OnShutdown to be called
rootGroup.shutdown();
stateManagerProvider.shutdown();
// invoke any methods annotated with @OnShutdown on Controller Services
for (final ControllerServiceNode serviceNode : getAllControllerServices()) {
try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
@ -1443,7 +1477,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
for (final Map.Entry<String, String> entry : controllerServiceDTO.getProperties().entrySet()) {
if (entry.getValue() != null) {
serviceNode.setProperty(entry.getKey(), entry.getValue());
serviceNode.setProperty(entry.getKey(), entry.getValue(), true);
}
}
}
@ -1561,7 +1595,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
if (config.getProperties() != null) {
for (final Map.Entry<String, String> entry : config.getProperties().entrySet()) {
if (entry.getValue() != null) {
procNode.setProperty(entry.getKey(), entry.getValue());
procNode.setProperty(entry.getKey(), entry.getValue(), true);
}
}
}
@ -3019,7 +3053,6 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
*
* @param clustered true if clustered
* @param clusterInstanceId if clustered is true, indicates the InstanceID of the Cluster Manager
* @param clusterManagerDn the DN of the NCM
*/
public void setClustered(final boolean clustered, final String clusterInstanceId, final String clusterManagerDn) {
writeLock.lock();
@ -3046,8 +3079,26 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
if (clustered) {
nodeBulletinSubscriber.set(new NodeBulletinProcessingStrategy());
bulletinRepository.overrideDefaultBulletinProcessing(nodeBulletinSubscriber.get());
stateManagerProvider.enableClusterProvider();
if (zooKeeperStateServer != null) {
processScheduler.submitFrameworkTask(new Runnable() {
@Override
public void run() {
try {
zooKeeperStateServer.start();
} catch (final Exception e) {
LOG.error("NiFi was connected to the cluster but failed to start embedded ZooKeeper Server", e);
}
}
});
}
} else {
bulletinRepository.restoreDefaultBulletinProcessing();
if (zooKeeperStateServer != null) {
zooKeeperStateServer.shutdown();
}
stateManagerProvider.disableClusterProvider();
}
final List<RemoteProcessGroup> remoteGroups = getGroup(getRootGroupId()).findAllRemoteProcessGroups();
@ -3063,6 +3114,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
}
}
/**
* @return true if this instance is the primary node in the cluster; false otherwise
*/
@ -3687,8 +3739,6 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
hbPayload.setCounters(getCounters());
hbPayload.setSystemDiagnostics(getSystemDiagnostics());
hbPayload.setProcessGroupStatus(procGroupStatus);
hbPayload.setSiteToSitePort(remoteInputSocketPort);
hbPayload.setSiteToSiteSecure(isSiteToSiteSecure);
// create heartbeat message
final Heartbeat heartbeat = new Heartbeat(getNodeId(), bean.isPrimary(), bean.isConnected(), hbPayload.marshal());

View File

@ -26,7 +26,9 @@ import java.nio.file.Path;
import java.nio.file.Paths;
import java.nio.file.StandardOpenOption;
import java.util.Calendar;
import java.util.Collections;
import java.util.Date;
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
@ -37,6 +39,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.zip.GZIPInputStream;
import java.util.zip.GZIPOutputStream;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.cluster.ConnectionException;
import org.apache.nifi.cluster.protocol.ConnectionRequest;
import org.apache.nifi.cluster.protocol.ConnectionResponse;
@ -57,9 +60,11 @@ import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
import org.apache.nifi.cluster.protocol.message.ReconnectionFailureMessage;
import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage;
import org.apache.nifi.cluster.protocol.message.ReconnectionResponseMessage;
import org.apache.nifi.components.state.Scope;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.encrypt.StringEncryptor;
import org.apache.nifi.engine.FlowEngine;
import org.apache.nifi.events.BulletinFactory;
import org.apache.nifi.util.file.FileUtils;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.lifecycle.LifeCycleStartException;
import org.apache.nifi.logging.LogLevel;
@ -69,15 +74,17 @@ import org.apache.nifi.reporting.Bulletin;
import org.apache.nifi.services.FlowService;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.util.NiFiProperties;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.encrypt.StringEncryptor;
import org.apache.nifi.util.file.FileUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class StandardFlowService implements FlowService, ProtocolHandler {
private static final String EVENT_CATEGORY = "Controller";
private static final String CLUSTER_NODE_CONFIG = "Cluster Node Configuration";
// state keys
private static final String NODE_UUID = "Node UUID";
private final FlowController controller;
private final Path flowXml;
@ -169,8 +176,21 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
final InetSocketAddress nodeApiAddress = properties.getNodeApiAddress();
final InetSocketAddress nodeSocketAddress = properties.getClusterNodeProtocolAddress();
String nodeUuid = null;
final StateManager stateManager = controller.getStateManagerProvider().getStateManager(CLUSTER_NODE_CONFIG);
if (stateManager != null) {
nodeUuid = stateManager.getState(Scope.LOCAL).get(NODE_UUID);
}
if (nodeUuid == null) {
nodeUuid = UUID.randomUUID().toString();
}
// use a random UUID as the proposed node identifier
this.nodeId = new NodeIdentifier(UUID.randomUUID().toString(), nodeApiAddress.getHostName(), nodeApiAddress.getPort(), nodeSocketAddress.getHostName(), nodeSocketAddress.getPort());
this.nodeId = new NodeIdentifier(nodeUuid,
nodeApiAddress.getHostName(), nodeApiAddress.getPort(),
nodeSocketAddress.getHostName(), nodeSocketAddress.getPort(),
properties.getRemoteInputHost(), properties.getRemoteInputPort(), properties.isSiteToSiteSecure());
} else {
this.configuredForClustering = false;
@ -179,6 +199,7 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
}
@Override
public void saveFlowChanges() throws IOException {
writeLock.lock();
@ -616,8 +637,6 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
if (firstControllerInitialization) {
logger.debug("First controller initialization. Loading reporting tasks and initializing controller.");
// load the controller tasks
// dao.loadReportingTasks(controller);
// initialize the flow
controller.initializeFlow();
@ -650,8 +669,8 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
for (int i = 0; i < maxAttempts || retryIndefinitely; i++) {
try {
response = senderListener.requestConnection(requestMsg).getConnectionResponse();
if (response.isBlockedByFirewall()) {
logger.warn("Connection request was blocked by cluster manager's firewall.");
if (response.getRejectionReason() != null) {
logger.warn("Connection request was blocked by cluster manager with the explanation: " + response.getRejectionReason());
// set response to null and treat a firewall blockage the same as getting no response from manager
response = null;
break;
@ -667,7 +686,6 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
// we received a successful connection response from manager
break;
}
} catch (final Exception pe) {
// could not create a socket and communicate with manager
logger.warn("Failed to connect to cluster due to: " + pe, pe);
@ -691,6 +709,16 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
return null;
} else {
// cluster manager provided a successful response with a current dataflow
// persist node uuid and index returned by NCM and return the response to the caller
try {
// Ensure that we have registered our 'cluster node configuration' state key
final Map<String, String> map = Collections.singletonMap(NODE_UUID, response.getNodeIdentifier().getId());
controller.getStateManagerProvider().getStateManager(CLUSTER_NODE_CONFIG).setState(map, Scope.LOCAL);
} catch (final IOException ioe) {
logger.warn("Received successful response from Cluster Manager but failed to persist state about the Node's Unique Identifier and the Node's Index. "
+ "This node may be assigned a different UUID when the node is restarted.", ioe);
}
return response;
}
} finally {

View File

@ -171,7 +171,7 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
controller.setMaxEventDrivenThreadCount(maxThreadCount / 3);
}
final Element reportingTasksElement = (Element) DomUtils.getChild(rootElement, "reportingTasks");
final Element reportingTasksElement = DomUtils.getChild(rootElement, "reportingTasks");
final List<Element> taskElements;
if (reportingTasksElement == null) {
taskElements = Collections.emptyList();
@ -179,7 +179,7 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
taskElements = DomUtils.getChildElementsByTagName(reportingTasksElement, "reportingTask");
}
final Element controllerServicesElement = (Element) DomUtils.getChild(rootElement, "controllerServices");
final Element controllerServicesElement = DomUtils.getChild(rootElement, "controllerServices");
final List<Element> controllerServiceElements;
if (controllerServicesElement == null) {
controllerServiceElements = Collections.emptyList();
@ -252,7 +252,7 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
// get the root group XML element
final Element rootGroupElement = (Element) rootElement.getElementsByTagName("rootGroup").item(0);
final Element controllerServicesElement = (Element) DomUtils.getChild(rootElement, "controllerServices");
final Element controllerServicesElement = DomUtils.getChild(rootElement, "controllerServices");
if (controllerServicesElement != null) {
final List<Element> serviceElements = DomUtils.getChildElementsByTagName(controllerServicesElement, "controllerService");
@ -274,7 +274,7 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
updateProcessGroup(controller, /* parent group */ null, rootGroupElement, encryptor);
}
final Element reportingTasksElement = (Element) DomUtils.getChild(rootElement, "reportingTasks");
final Element reportingTasksElement = DomUtils.getChild(rootElement, "reportingTasks");
if (reportingTasksElement != null) {
final List<Element> taskElements = DomUtils.getChildElementsByTagName(reportingTasksElement, "reportingTask");
for (final Element taskElement : taskElements) {
@ -403,9 +403,9 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
for (final Map.Entry<String, String> entry : dto.getProperties().entrySet()) {
if (entry.getValue() == null) {
reportingTask.removeProperty(entry.getKey());
reportingTask.removeProperty(entry.getKey(), false);
} else {
reportingTask.setProperty(entry.getKey(), entry.getValue());
reportingTask.setProperty(entry.getKey(), entry.getValue(), false);
}
}
@ -735,9 +735,9 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
for (final Map.Entry<String, String> entry : config.getProperties().entrySet()) {
if (entry.getValue() == null) {
procNode.removeProperty(entry.getKey());
procNode.removeProperty(entry.getKey(), false);
} else {
procNode.setProperty(entry.getKey(), entry.getValue());
procNode.setProperty(entry.getKey(), entry.getValue(), false);
}
}

View File

@ -125,15 +125,15 @@ public abstract class AbstractReportingTaskNode extends AbstractConfiguredCompon
}
@Override
public void setProperty(final String name, final String value) {
super.setProperty(name, value);
public void setProperty(final String name, final String value, final boolean triggerOnPropertyModified) {
super.setProperty(name, value, triggerOnPropertyModified);
onConfigured();
}
@Override
public boolean removeProperty(String name) {
final boolean removed = super.removeProperty(name);
public boolean removeProperty(String name, final boolean triggerOnPropertyModified) {
final boolean removed = super.removeProperty(name, triggerOnPropertyModified);
if (removed) {
onConfigured();
}

View File

@ -26,6 +26,7 @@ import org.apache.nifi.attribute.expression.language.Query;
import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.controller.ControllerServiceLookup;
@ -37,24 +38,27 @@ import org.apache.nifi.reporting.Bulletin;
import org.apache.nifi.reporting.BulletinRepository;
import org.apache.nifi.reporting.EventAccess;
import org.apache.nifi.reporting.ReportingContext;
import org.apache.nifi.reporting.ReportingTask;
import org.apache.nifi.reporting.Severity;
public class StandardReportingContext implements ReportingContext, ControllerServiceLookup {
private final FlowController flowController;
private final EventAccess eventAccess;
private final ReportingTask reportingTask;
private final BulletinRepository bulletinRepository;
private final ControllerServiceProvider serviceProvider;
private final Map<PropertyDescriptor, String> properties;
private final Map<PropertyDescriptor, PreparedQuery> preparedQueries;
public StandardReportingContext(final FlowController flowController, final BulletinRepository bulletinRepository,
final Map<PropertyDescriptor, String> properties, final ControllerServiceProvider serviceProvider) {
final Map<PropertyDescriptor, String> properties, final ControllerServiceProvider serviceProvider, final ReportingTask reportingTask) {
this.flowController = flowController;
this.eventAccess = flowController;
this.bulletinRepository = bulletinRepository;
this.properties = Collections.unmodifiableMap(properties);
this.serviceProvider = serviceProvider;
this.reportingTask = reportingTask;
preparedQueries = new HashMap<>();
for (final Map.Entry<PropertyDescriptor, String> entry : properties.entrySet()) {
@ -140,4 +144,8 @@ public class StandardReportingContext implements ReportingContext, ControllerSer
return serviceProvider.getControllerServiceName(serviceIdentifier);
}
@Override
public StateManager getStateManager() {
return flowController.getStateManagerProvider().getStateManager(reportingTask.getIdentifier());
}
}

View File

@ -35,6 +35,6 @@ public class StandardReportingTaskNode extends AbstractReportingTaskNode impleme
@Override
public ReportingContext getReportingContext() {
return new StandardReportingContext(flowController, flowController.getBulletinRepository(), getProperties(), flowController);
return new StandardReportingContext(flowController, flowController.getBulletinRepository(), getProperties(), flowController, getReportingTask());
}
}

View File

@ -49,8 +49,8 @@ public class BatchingSessionFactory implements ProcessSessionFactory {
return highThroughputSession;
}
private class HighThroughputSession implements ProcessSession {
private class HighThroughputSession implements ProcessSession {
private final StandardProcessSession session;
public HighThroughputSession(final StandardProcessSession session) {
@ -241,7 +241,5 @@ public class BatchingSessionFactory implements ProcessSessionFactory {
public ProvenanceReporter getProvenanceReporter() {
return session.getProvenanceReporter();
}
}
}

View File

@ -26,6 +26,7 @@ import java.util.concurrent.TimeUnit;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.connectable.Connection;
import org.apache.nifi.controller.ControllerService;
@ -46,10 +47,12 @@ public class ConnectableProcessContext implements ProcessContext {
private final Connectable connectable;
private final StringEncryptor encryptor;
private final StateManager stateManager;
public ConnectableProcessContext(final Connectable connectable, final StringEncryptor encryptor) {
public ConnectableProcessContext(final Connectable connectable, final StringEncryptor encryptor, final StateManager stateManager) {
this.connectable = connectable;
this.encryptor = encryptor;
this.stateManager = stateManager;
}
@Override
@ -235,4 +238,9 @@ public class ConnectableProcessContext implements ProcessContext {
public boolean isExpressionLanguagePresent(PropertyDescriptor property) {
return false;
}
@Override
public StateManager getStateManager() {
return stateManager;
}
}

View File

@ -24,6 +24,8 @@ import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.components.state.StateManagerProvider;
import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.controller.EventBasedWorker;
import org.apache.nifi.controller.EventDrivenWorkerQueue;
@ -54,7 +56,8 @@ public class EventDrivenSchedulingAgent implements SchedulingAgent {
private static final Logger logger = LoggerFactory.getLogger(EventDrivenSchedulingAgent.class);
private final FlowEngine flowEngine;
private final ControllerServiceProvider controllerServiceProvider;
private final ControllerServiceProvider serviceProvider;
private final StateManagerProvider stateManagerProvider;
private final EventDrivenWorkerQueue workerQueue;
private final ProcessContextFactory contextFactory;
private final AtomicInteger maxThreadCount;
@ -65,10 +68,11 @@ public class EventDrivenSchedulingAgent implements SchedulingAgent {
private final ConcurrentMap<Connectable, AtomicLong> connectionIndexMap = new ConcurrentHashMap<>();
private final ConcurrentMap<Connectable, ScheduleState> scheduleStates = new ConcurrentHashMap<>();
public EventDrivenSchedulingAgent(final FlowEngine flowEngine, final ControllerServiceProvider flowController,
public EventDrivenSchedulingAgent(final FlowEngine flowEngine, final ControllerServiceProvider serviceProvider, final StateManagerProvider stateManagerProvider,
final EventDrivenWorkerQueue workerQueue, final ProcessContextFactory contextFactory, final int maxThreadCount, final StringEncryptor encryptor) {
this.flowEngine = flowEngine;
this.controllerServiceProvider = flowController;
this.serviceProvider = serviceProvider;
this.stateManagerProvider = stateManagerProvider;
this.workerQueue = workerQueue;
this.contextFactory = contextFactory;
this.maxThreadCount = new AtomicInteger(maxThreadCount);
@ -80,6 +84,10 @@ public class EventDrivenSchedulingAgent implements SchedulingAgent {
}
}
private StateManager getStateManager(final String componentId) {
return stateManagerProvider.getStateManager(componentId);
}
@Override
public void shutdown() {
flowEngine.shutdown();
@ -177,7 +185,8 @@ public class EventDrivenSchedulingAgent implements SchedulingAgent {
if (connectable instanceof ProcessorNode) {
final ProcessorNode procNode = (ProcessorNode) connectable;
final StandardProcessContext standardProcessContext = new StandardProcessContext(procNode, controllerServiceProvider, encryptor);
final StandardProcessContext standardProcessContext = new StandardProcessContext(procNode, serviceProvider,
encryptor, getStateManager(connectable.getIdentifier()));
final long runNanos = procNode.getRunDuration(TimeUnit.NANOSECONDS);
final ProcessSessionFactory sessionFactory;
@ -251,7 +260,7 @@ public class EventDrivenSchedulingAgent implements SchedulingAgent {
}
} else {
final ProcessSessionFactory sessionFactory = new StandardProcessSessionFactory(context);
final ConnectableProcessContext connectableProcessContext = new ConnectableProcessContext(connectable, encryptor);
final ConnectableProcessContext connectableProcessContext = new ConnectableProcessContext(connectable, encryptor, getStateManager(connectable.getIdentifier()));
trigger(connectable, scheduleState, connectableProcessContext, sessionFactory);
// See explanation above for the ProcessorNode as to why we do this.

View File

@ -25,6 +25,7 @@ import java.util.concurrent.Callable;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.connectable.ConnectableType;
import org.apache.nifi.controller.FlowController;
@ -61,6 +62,10 @@ public class QuartzSchedulingAgent implements SchedulingAgent {
this.encryptor = enryptor;
}
private StateManager getStateManager(final String componentId) {
return flowController.getStateManagerProvider().getStateManager(componentId);
}
@Override
public void shutdown() {
}
@ -133,14 +138,15 @@ public class QuartzSchedulingAgent implements SchedulingAgent {
final List<AtomicBoolean> triggers = new ArrayList<>();
for (int i = 0; i < connectable.getMaxConcurrentTasks(); i++) {
final Callable<Boolean> continuallyRunTask;
if (connectable.getConnectableType() == ConnectableType.PROCESSOR) {
final ProcessorNode procNode = (ProcessorNode) connectable;
final StandardProcessContext standardProcContext = new StandardProcessContext(procNode, flowController, encryptor);
final StandardProcessContext standardProcContext = new StandardProcessContext(procNode, flowController, encryptor, getStateManager(connectable.getIdentifier()));
ContinuallyRunProcessorTask runnableTask = new ContinuallyRunProcessorTask(this, procNode, flowController, contextFactory, scheduleState, standardProcContext);
continuallyRunTask = runnableTask;
} else {
final ConnectableProcessContext connProcContext = new ConnectableProcessContext(connectable, encryptor);
final ConnectableProcessContext connProcContext = new ConnectableProcessContext(connectable, encryptor, getStateManager(connectable.getIdentifier()));
continuallyRunTask = new ContinuallyRunConnectableTask(contextFactory, connectable, scheduleState, connProcContext);
}

View File

@ -24,6 +24,8 @@ import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.nifi.annotation.lifecycle.OnStopped;
public class ScheduleState {
private final AtomicInteger activeThreadCount = new AtomicInteger(0);
@ -62,9 +64,9 @@ public class ScheduleState {
}
/**
* Maintains an AtomicBoolean so that the first thread to call this method after a Processor is no longer scheduled to run will receive a <code>true</code> and MUST call the methods annotated with
*
* @OnStopped
* Maintains an AtomicBoolean so that the first thread to call this method after a Processor is no longer
* scheduled to run will receive a <code>true</code> and MUST call the methods annotated with
* {@link OnStopped @OnStopped}
*
* @return <code>true</code> if the caller is required to call Processor methods annotated with
* @OnStopped, <code>false</code> otherwise

View File

@ -32,6 +32,8 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.components.state.StateManagerProvider;
import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.connectable.Funnel;
import org.apache.nifi.connectable.Port;
@ -74,6 +76,7 @@ public final class StandardProcessScheduler implements ProcessScheduler {
private final Heartbeater heartbeater;
private final long administrativeYieldMillis;
private final String administrativeYieldDuration;
private final StateManagerProvider stateManagerProvider;
private final ConcurrentMap<Object, ScheduleState> scheduleStates = new ConcurrentHashMap<>();
private final ScheduledExecutorService frameworkTaskExecutor;
@ -84,10 +87,12 @@ public final class StandardProcessScheduler implements ProcessScheduler {
private final StringEncryptor encryptor;
public StandardProcessScheduler(final Heartbeater heartbeater, final ControllerServiceProvider controllerServiceProvider, final StringEncryptor encryptor) {
public StandardProcessScheduler(final Heartbeater heartbeater, final ControllerServiceProvider controllerServiceProvider, final StringEncryptor encryptor,
final StateManagerProvider stateManagerProvider) {
this.heartbeater = heartbeater;
this.controllerServiceProvider = controllerServiceProvider;
this.encryptor = encryptor;
this.stateManagerProvider = stateManagerProvider;
administrativeYieldDuration = NiFiProperties.getInstance().getAdministrativeYieldDuration();
administrativeYieldMillis = FormatUtils.getTimeDuration(administrativeYieldDuration, TimeUnit.MILLISECONDS);
@ -95,6 +100,10 @@ public final class StandardProcessScheduler implements ProcessScheduler {
frameworkTaskExecutor = new FlowEngine(4, "Framework Task Thread");
}
private StateManager getStateManager(final String componentId) {
return stateManagerProvider.getStateManager(componentId);
}
public void scheduleFrameworkTask(final Runnable command, final String taskName, final long initialDelay, final long delay, final TimeUnit timeUnit) {
frameworkTaskExecutor.scheduleWithFixedDelay(new Runnable() {
@Override
@ -102,7 +111,7 @@ public final class StandardProcessScheduler implements ProcessScheduler {
try {
command.run();
} catch (final Throwable t) {
LOG.error("Failed to run Framework Task {} due to {}", command, t.toString());
LOG.error("Failed to run Framework Task {} due to {}", taskName, t.toString());
if (LOG.isDebugEnabled()) {
LOG.error("", t);
}
@ -111,6 +120,15 @@ public final class StandardProcessScheduler implements ProcessScheduler {
}, initialDelay, delay, timeUnit);
}
/**
* Submits the given task to be executed exactly once in a background thread
*
* @param task the task to perform
*/
public void submitFrameworkTask(final Runnable task) {
frameworkTaskExecutor.submit(task);
}
@Override
public void setMaxThreadCount(final SchedulingStrategy schedulingStrategy, final int maxThreadCount) {
final SchedulingAgent agent = getSchedulingAgent(schedulingStrategy);
@ -299,7 +317,7 @@ public final class StandardProcessScheduler implements ProcessScheduler {
public void run() {
try (final NarCloseable x = NarCloseable.withNarLoader()) {
final long lastStopTime = scheduleState.getLastStopTime();
final StandardProcessContext processContext = new StandardProcessContext(procNode, controllerServiceProvider, encryptor);
final StandardProcessContext processContext = new StandardProcessContext(procNode, controllerServiceProvider, encryptor, getStateManager(procNode.getIdentifier()));
final Set<String> serviceIds = new HashSet<>();
for (final PropertyDescriptor descriptor : processContext.getProperties().keySet()) {
@ -343,7 +361,8 @@ public final class StandardProcessScheduler implements ProcessScheduler {
return;
}
final SchedulingContext schedulingContext = new StandardSchedulingContext(processContext, controllerServiceProvider, procNode);
final SchedulingContext schedulingContext = new StandardSchedulingContext(processContext, controllerServiceProvider,
procNode, getStateManager(procNode.getIdentifier()));
ReflectionUtils.invokeMethodsWithAnnotations(OnScheduled.class, org.apache.nifi.processor.annotation.OnScheduled.class, procNode.getProcessor(), schedulingContext);
getSchedulingAgent(procNode).schedule(procNode, scheduleState);
@ -420,7 +439,7 @@ public final class StandardProcessScheduler implements ProcessScheduler {
@Override
public void run() {
try (final NarCloseable x = NarCloseable.withNarLoader()) {
final StandardProcessContext processContext = new StandardProcessContext(procNode, controllerServiceProvider, encryptor);
final StandardProcessContext processContext = new StandardProcessContext(procNode, controllerServiceProvider, encryptor, getStateManager(procNode.getIdentifier()));
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnUnscheduled.class, procNode.getProcessor(), processContext);
@ -503,7 +522,7 @@ public final class StandardProcessScheduler implements ProcessScheduler {
getSchedulingAgent(connectable).unschedule(connectable, state);
if (!state.isScheduled() && state.getActiveThreadCount() == 0 && state.mustCallOnStoppedMethods()) {
final ConnectableProcessContext processContext = new ConnectableProcessContext(connectable, encryptor);
final ConnectableProcessContext processContext = new ConnectableProcessContext(connectable, encryptor, getStateManager(connectable.getIdentifier()));
try (final NarCloseable x = NarCloseable.withNarLoader()) {
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, connectable, processContext);
heartbeater.heartbeat();

View File

@ -23,6 +23,7 @@ import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.connectable.ConnectableType;
import org.apache.nifi.controller.FlowController;
@ -67,6 +68,10 @@ public class TimerDrivenSchedulingAgent implements SchedulingAgent {
}
}
private StateManager getStateManager(final String componentId) {
return flowController.getStateManagerProvider().getStateManager(componentId);
}
@Override
public void shutdown() {
flowEngine.shutdown();
@ -96,14 +101,14 @@ public class TimerDrivenSchedulingAgent implements SchedulingAgent {
// Determine the task to run and create it.
if (connectable.getConnectableType() == ConnectableType.PROCESSOR) {
final ProcessorNode procNode = (ProcessorNode) connectable;
final StandardProcessContext standardProcContext = new StandardProcessContext(procNode, flowController, encryptor);
final StandardProcessContext standardProcContext = new StandardProcessContext(procNode, flowController, encryptor, getStateManager(connectable.getIdentifier()));
final ContinuallyRunProcessorTask runnableTask = new ContinuallyRunProcessorTask(this, procNode, flowController,
contextFactory, scheduleState, standardProcContext);
continuallyRunTask = runnableTask;
processContext = standardProcContext;
} else {
processContext = new ConnectableProcessContext(connectable, encryptor);
processContext = new ConnectableProcessContext(connectable, encryptor, getStateManager(connectable.getIdentifier()));
continuallyRunTask = new ContinuallyRunConnectableTask(contextFactory, connectable, scheduleState, processContext);
}

View File

@ -155,9 +155,9 @@ public class ControllerServiceLoader {
for (final Map.Entry<String, String> entry : dto.getProperties().entrySet()) {
if (entry.getValue() == null) {
node.removeProperty(entry.getKey());
node.removeProperty(entry.getKey(), false);
} else {
node.setProperty(entry.getKey(), entry.getValue());
node.setProperty(entry.getKey(), entry.getValue(), false);
}
}
}

View File

@ -18,6 +18,7 @@ package org.apache.nifi.controller.service;
import java.util.Set;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.controller.ControllerServiceInitializationContext;
import org.apache.nifi.controller.ControllerServiceLookup;
@ -28,11 +29,13 @@ public class StandardControllerServiceInitializationContext implements Controlle
private final String id;
private final ControllerServiceProvider serviceProvider;
private final ComponentLog logger;
private final StateManager stateManager;
public StandardControllerServiceInitializationContext(final String identifier, final ComponentLog logger, final ControllerServiceProvider serviceProvider) {
public StandardControllerServiceInitializationContext(final String identifier, final ComponentLog logger, final ControllerServiceProvider serviceProvider, final StateManager stateManager) {
this.id = identifier;
this.logger = logger;
this.serviceProvider = serviceProvider;
this.stateManager = stateManager;
}
@Override
@ -79,4 +82,9 @@ public class StandardControllerServiceInitializationContext implements Controlle
public ComponentLog getLogger() {
return logger;
}
@Override
public StateManager getStateManager() {
return stateManager;
}
}

View File

@ -123,14 +123,14 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
}
@Override
public void setProperty(final String name, final String value) {
super.setProperty(name, value);
public void setProperty(final String name, final String value, final boolean triggerOnPropertyModified) {
super.setProperty(name, value, triggerOnPropertyModified);
onConfigured();
}
@Override
public boolean removeProperty(String name) {
final boolean removed = super.removeProperty(name);
public boolean removeProperty(String name, final boolean triggerOnPropertyModified) {
final boolean removed = super.removeProperty(name, triggerOnPropertyModified);
if (removed) {
onConfigured();
}

View File

@ -38,6 +38,8 @@ import java.util.concurrent.Executors;
import org.apache.nifi.annotation.lifecycle.OnAdded;
import org.apache.nifi.annotation.lifecycle.OnRemoved;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.components.state.StateManagerProvider;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.controller.ConfiguredComponent;
import org.apache.nifi.controller.ControllerService;
@ -46,8 +48,8 @@ import org.apache.nifi.controller.ProcessorNode;
import org.apache.nifi.controller.ReportingTaskNode;
import org.apache.nifi.controller.ScheduledState;
import org.apache.nifi.controller.ValidationContextFactory;
import org.apache.nifi.controller.exception.ControllerServiceInstantiationException;
import org.apache.nifi.controller.exception.ComponentLifeCycleException;
import org.apache.nifi.controller.exception.ControllerServiceInstantiationException;
import org.apache.nifi.events.BulletinFactory;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.nar.ExtensionManager;
@ -69,6 +71,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
private final ConcurrentMap<String, ControllerServiceNode> controllerServices;
private static final Set<Method> validDisabledMethods;
private final BulletinRepository bulletinRepo;
private final StateManagerProvider stateManagerProvider;
static {
// methods that are okay to be called when the service is disabled.
@ -82,12 +85,13 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
validDisabledMethods = Collections.unmodifiableSet(validMethods);
}
public StandardControllerServiceProvider(final ProcessScheduler scheduler, final BulletinRepository bulletinRepo) {
public StandardControllerServiceProvider(final ProcessScheduler scheduler, final BulletinRepository bulletinRepo, final StateManagerProvider stateManagerProvider) {
// the following 2 maps must be updated atomically, but we do not lock around them because they are modified
// only in the createControllerService method, and both are modified before the method returns
this.controllerServices = new ConcurrentHashMap<>();
this.processScheduler = scheduler;
this.bulletinRepo = bulletinRepo;
this.stateManagerProvider = stateManagerProvider;
}
private Class<?>[] getInterfaces(final Class<?> cls) {
@ -110,6 +114,10 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
}
}
private StateManager getStateManager(final String componentId) {
return stateManagerProvider.getStateManager(componentId);
}
@Override
public ControllerServiceNode createControllerService(final String type, final String id, final boolean firstTimeAdded) {
if (type == null || id == null) {
@ -171,7 +179,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
logger.info("Created Controller Service of type {} with identifier {}", type, id);
final ComponentLog serviceLogger = new SimpleProcessLogger(id, originalService);
originalService.initialize(new StandardControllerServiceInitializationContext(id, serviceLogger, this));
originalService.initialize(new StandardControllerServiceInitializationContext(id, serviceLogger, this, getStateManager(id)));
final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(this);
@ -489,6 +497,8 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
}
controllerServices.remove(serviceNode.getIdentifier());
stateManagerProvider.onComponentRemoved(serviceNode.getIdentifier());
}
@Override

View File

@ -0,0 +1,38 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.state;
import java.util.Set;
public interface ClusterState {
/**
* @return <code>true</code> if this instance of NiFi is connected to a cluster, <code>false</code> if the node is disconnected
*/
boolean isConnected();
/**
* @return the identifier that is used to identify this node in the cluster
*/
String getNodeIdentifier();
/**
* @return a Set of {@link NodeDescription} objects that can be used to determine which other nodes are in the same cluster. This
* Set will not be <code>null</code> but will be empty if the node is not connected to a cluster
*/
Set<NodeDescription> getNodes();
}

View File

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

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.controller.state;
/**
* Provides information about a node in a NiFi cluster
*/
public interface NodeDescription {
/**
* @return the unique identifier for this node in the cluster
*/
String getNodeIdentifier();
/**
* @return the hostname of the node
*/
String getHostname();
/**
* @return the port on which the node's embedded ZooKeeper Server is running, or <code>null</code> if the node is
* not running an embedded ZooKeeper server
*/
Integer getEmbeddedZooKeeperPort();
}

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.controller.state;
import java.io.IOException;
import java.util.Map;
import org.apache.nifi.components.state.Scope;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.components.state.StateMap;
import org.apache.nifi.components.state.StateProvider;
public class StandardStateManager implements StateManager {
private final StateProvider localProvider;
private final StateProvider clusterProvider;
private final String componentId;
public StandardStateManager(final StateProvider localProvider, final StateProvider clusterProvider, final String componentId) {
this.localProvider = localProvider;
this.clusterProvider = clusterProvider;
this.componentId = componentId;
}
private StateProvider getProvider(final Scope scope) {
if (scope == Scope.LOCAL || clusterProvider == null || !clusterProvider.isEnabled()) {
return localProvider;
}
return clusterProvider;
}
@Override
public StateMap getState(final Scope scope) throws IOException {
return getProvider(scope).getState(componentId);
}
@Override
public boolean replace(final StateMap oldValue, final Map<String, String> newValue, final Scope scope) throws IOException {
return getProvider(scope).replace(oldValue, newValue, componentId);
}
@Override
public void setState(final Map<String, String> state, final Scope scope) throws IOException {
getProvider(scope).setState(state, componentId);
}
@Override
public void clear(final Scope scope) throws IOException {
getProvider(scope).clear(componentId);
}
@Override
public String toString() {
return "StandardStateManager[componentId=" + componentId + "]";
}
}

View File

@ -0,0 +1,49 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.state;
import java.util.Collections;
import java.util.Map;
import org.apache.nifi.components.state.StateMap;
public class StandardStateMap implements StateMap {
private final Map<String, String> stateValues;
private final long version;
public StandardStateMap(final Map<String, String> stateValues, final long version) {
this.stateValues = Collections.unmodifiableMap(stateValues == null ? Collections.<String, String> emptyMap() : stateValues);
this.version = version;
}
@Override
public long getVersion() {
return version;
}
@Override
public String get(final String key) {
return stateValues.get(key);
}
@Override
public Map<String, String> toMap() {
return stateValues;
}
}

View File

@ -0,0 +1,60 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.state;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import javax.net.ssl.SSLContext;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.components.state.StateProviderInitializationContext;
public class StandardStateProviderInitializationContext implements StateProviderInitializationContext {
private final String id;
private final Map<PropertyDescriptor, PropertyValue> properties;
private final SSLContext sslContext;
public StandardStateProviderInitializationContext(final String identifier, final Map<PropertyDescriptor, PropertyValue> properties, final SSLContext sslContext) {
this.id = identifier;
this.properties = new HashMap<>(properties);
this.sslContext = sslContext;
}
@Override
public Map<PropertyDescriptor, PropertyValue> getProperties() {
return Collections.unmodifiableMap(properties);
}
@Override
public PropertyValue getProperty(final PropertyDescriptor property) {
return properties.get(property);
}
@Override
public String getIdentifier() {
return id;
}
@Override
public SSLContext getSSLContext() {
return sslContext;
}
}

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.controller.state;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import org.apache.nifi.components.state.StateMap;
import org.wali.SerDe;
import org.wali.UpdateType;
public class StateMapSerDe implements SerDe<StateMapUpdate> {
private static final int VERSION = 0;
@Override
public void serializeEdit(final StateMapUpdate previousRecordState, final StateMapUpdate newRecordState, final DataOutputStream out) throws IOException {
serializeRecord(newRecordState, out);
}
@Override
public void serializeRecord(final StateMapUpdate record, final DataOutputStream out) throws IOException {
out.writeUTF(record.getComponentId());
out.writeUTF(record.getUpdateType().name());
if (record.getUpdateType() == UpdateType.DELETE) {
return;
}
final StateMap stateMap = record.getStateMap();
final long recordVersion = stateMap.getVersion();
out.writeLong(recordVersion);
final Map<String, String> map = stateMap.toMap();
out.writeInt(map.size());
for (final Map.Entry<String, String> entry : map.entrySet()) {
out.writeUTF(entry.getKey());
out.writeUTF(entry.getValue());
}
}
@Override
public StateMapUpdate deserializeEdit(final DataInputStream in, final Map<Object, StateMapUpdate> currentRecordStates, final int version) throws IOException {
return deserializeRecord(in, version);
}
@Override
public StateMapUpdate deserializeRecord(final DataInputStream in, final int version) throws IOException {
final String componentId = in.readUTF();
final String updateTypeName = in.readUTF();
final UpdateType updateType = UpdateType.valueOf(updateTypeName);
if (updateType == UpdateType.DELETE) {
return new StateMapUpdate(null, componentId, updateType);
}
final long recordVersion = in.readLong();
final int numEntries = in.readInt();
final Map<String, String> stateValues = new HashMap<>(numEntries);
for (int i = 0; i < numEntries; i++) {
final String key = in.readUTF();
final String value = in.readUTF();
stateValues.put(key, value);
}
return new StateMapUpdate(new StandardStateMap(stateValues, recordVersion), componentId, updateType);
}
@Override
public Object getRecordIdentifier(final StateMapUpdate record) {
return record.getComponentId();
}
@Override
public UpdateType getUpdateType(final StateMapUpdate record) {
return record.getUpdateType();
}
@Override
public String getLocation(final StateMapUpdate record) {
return null;
}
@Override
public int getVersion() {
return VERSION;
}
}

View File

@ -0,0 +1,45 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.state;
import org.apache.nifi.components.state.StateMap;
import org.wali.UpdateType;
public class StateMapUpdate {
private final StateMap stateMap;
private final String componentId;
private final UpdateType updateType;
public StateMapUpdate(final StateMap stateMap, final String componentId, final UpdateType updateType) {
this.stateMap = stateMap;
this.componentId = componentId;
this.updateType = updateType;
}
public StateMap getStateMap() {
return stateMap;
}
public String getComponentId() {
return componentId;
}
public UpdateType getUpdateType() {
return updateType;
}
}

View File

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

View File

@ -0,0 +1,142 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.state.config;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import javax.xml.parsers.DocumentBuilder;
import javax.xml.parsers.DocumentBuilderFactory;
import javax.xml.parsers.ParserConfigurationException;
import org.apache.nifi.controller.state.ConfigParseException;
import org.apache.nifi.util.DomUtils;
import org.w3c.dom.Document;
import org.w3c.dom.Element;
import org.xml.sax.SAXException;
public class StateManagerConfiguration {
private final Map<String, StateProviderConfiguration> providers;
private StateManagerConfiguration(final Map<String, StateProviderConfiguration> providerConfigs) {
this.providers = providerConfigs;
}
public Map<String, StateProviderConfiguration> getStateProviderConfigurations() {
return Collections.unmodifiableMap(providers);
}
public StateProviderConfiguration getStateProviderConfiguration(final String providerId) {
return providers.get(providerId);
}
public List<StateProviderConfiguration> getStateProviderConfigurations(final StateProviderScope scope) {
final List<StateProviderConfiguration> configs = new ArrayList<>();
for (final StateProviderConfiguration config : providers.values()) {
if (config.getScope() == scope) {
configs.add(config);
}
}
return configs;
}
public static StateManagerConfiguration parse(final File configFile) throws IOException, ConfigParseException {
final DocumentBuilderFactory factory = DocumentBuilderFactory.newInstance();
factory.setNamespaceAware(false);
final Document document;
DocumentBuilder builder;
try {
builder = factory.newDocumentBuilder();
document = builder.parse(configFile);
} catch (ParserConfigurationException | SAXException e) {
throw new ConfigParseException("Unable to parse file " + configFile + ", as it does not appear to be a valid XML File", e);
}
final Element rootElement = document.getDocumentElement();
final List<Element> localProviderElements = DomUtils.getChildElementsByTagName(rootElement, "local-provider");
if (localProviderElements.isEmpty()) {
throw new ConfigParseException("State Management config file " + configFile + " is not a valid configuration file, "
+ "as it does not contain a 'local-provider' element, or the local-provider element is not the child of the root element");
}
final Map<String, StateProviderConfiguration> configs = new HashMap<>();
for (final Element localProviderElement : localProviderElements) {
final StateProviderConfiguration providerConfig = parseProviderConfiguration(localProviderElement, StateProviderScope.LOCAL, configFile);
if (configs.containsKey(providerConfig.getId())) {
throw new ConfigParseException("State Management config file " + configFile + " is not a valid configuration file, "
+ "as it contains multiple providers with the \"id\" of \"" + providerConfig.getId() + "\"");
}
configs.put(providerConfig.getId(), providerConfig);
}
final List<Element> clusterProviderElements = DomUtils.getChildElementsByTagName(rootElement, "cluster-provider");
for (final Element clusterProviderElement : clusterProviderElements) {
final StateProviderConfiguration providerConfig = parseProviderConfiguration(clusterProviderElement, StateProviderScope.CLUSTER, configFile);
if (configs.containsKey(providerConfig.getId())) {
throw new ConfigParseException("State Management config file " + configFile + " is not a valid configuration file, "
+ "as it contains multiple providers with the \"id\" of \"" + providerConfig.getId() + "\"");
}
configs.put(providerConfig.getId(), providerConfig);
}
return new StateManagerConfiguration(configs);
}
private static StateProviderConfiguration parseProviderConfiguration(final Element providerElement, final StateProviderScope scope, final File configFile) throws ConfigParseException {
final String elementName = providerElement.getNodeName();
final String id = DomUtils.getChildText(providerElement, "id");
if (id == null) {
throw new ConfigParseException("State Management config file " + configFile + " is not a valid configuration file, "
+ "as a " + elementName + " element does not contain an \"id\" element");
}
if (id.trim().isEmpty()) {
throw new ConfigParseException("State Management config file " + configFile + " is not a valid configuration file, "
+ "as a " + elementName + "'s \"id\" element is empty");
}
final String className = DomUtils.getChildText(providerElement, "class");
if (className == null) {
throw new ConfigParseException("State Management config file " + configFile + " is not a valid configuration file, "
+ "as a " + elementName + " element does not contain an \"class\" element");
}
if (className.trim().isEmpty()) {
throw new ConfigParseException("State Management config file " + configFile + " is not a valid configuration file, "
+ "as a " + elementName + "'s \"class\" element is empty");
}
final List<Element> propertyElements = DomUtils.getChildElementsByTagName(providerElement, "property");
final Map<String, String> propertyMap = new HashMap<>();
for (final Element propertyElement : propertyElements) {
final String propertyName = propertyElement.getAttribute("name");
final String propertyValue = propertyElement.getTextContent();
propertyMap.put(propertyName, propertyValue);
}
return new StateProviderConfiguration(id, className, scope, propertyMap);
}
}

View File

@ -0,0 +1,51 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.state.config;
import java.util.HashMap;
import java.util.Map;
public class StateProviderConfiguration {
private final String id;
private final StateProviderScope scope;
private final String className;
private final Map<String, String> properties;
public StateProviderConfiguration(final String id, final String className, final StateProviderScope scope, final Map<String, String> properties) {
this.id = id;
this.className = className;
this.scope = scope;
this.properties = new HashMap<>(properties);
}
public String getId() {
return id;
}
public String getClassName() {
return className;
}
public Map<String, String> getProperties() {
return properties;
}
public StateProviderScope getScope() {
return scope;
}
}

View File

@ -0,0 +1,30 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.state.config;
public enum StateProviderScope {
/**
* Provider is a Local State Provider
*/
LOCAL,
/**
* Provider is a Cluster State Provider
*/
CLUSTER;
}

View File

@ -0,0 +1,295 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.state.manager;
import java.io.File;
import java.io.IOException;
import java.util.Collection;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import javax.net.ssl.SSLContext;
import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.components.state.Scope;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.components.state.StateManagerProvider;
import org.apache.nifi.components.state.StateProvider;
import org.apache.nifi.components.state.StateProviderInitializationContext;
import org.apache.nifi.controller.state.ConfigParseException;
import org.apache.nifi.controller.state.StandardStateManager;
import org.apache.nifi.controller.state.StandardStateProviderInitializationContext;
import org.apache.nifi.controller.state.config.StateManagerConfiguration;
import org.apache.nifi.controller.state.config.StateProviderConfiguration;
import org.apache.nifi.controller.state.config.StateProviderScope;
import org.apache.nifi.framework.security.util.SslContextFactory;
import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.processor.StandardValidationContext;
import org.apache.nifi.util.NiFiProperties;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class StandardStateManagerProvider implements StateManagerProvider {
private static final Logger logger = LoggerFactory.getLogger(StandardStateManagerProvider.class);
private final ConcurrentMap<String, StateManager> stateManagers = new ConcurrentHashMap<>();
private final StateProvider localStateProvider;
private final StateProvider clusterStateProvider;
private StandardStateManagerProvider(final StateProvider localStateProvider, final StateProvider clusterStateProvider) {
this.localStateProvider = localStateProvider;
this.clusterStateProvider = clusterStateProvider;
}
public static StateManagerProvider create(final NiFiProperties properties) throws ConfigParseException, IOException {
final StateProvider localProvider = createLocalStateProvider(properties);
final StateProvider clusterProvider;
if (properties.isNode()) {
clusterProvider = createClusteredStateProvider(properties);
} else {
clusterProvider = null;
}
return new StandardStateManagerProvider(localProvider, clusterProvider);
}
private static StateProvider createLocalStateProvider(final NiFiProperties properties) throws IOException, ConfigParseException {
final File configFile = properties.getStateManagementConfigFile();
return createStateProvider(configFile, StateProviderScope.LOCAL, properties);
}
private static StateProvider createClusteredStateProvider(final NiFiProperties properties) throws IOException, ConfigParseException {
final File configFile = properties.getStateManagementConfigFile();
return createStateProvider(configFile, StateProviderScope.CLUSTER, properties);
}
private static StateProvider createStateProvider(final File configFile, final StateProviderScope scope, final NiFiProperties properties) throws ConfigParseException, IOException {
final String providerId;
final String providerIdPropertyName;
final String providerDescription;
final String providerXmlElementName;
final String oppositeScopeXmlElementName;
switch (scope) {
case CLUSTER:
providerId = properties.getClusterStateProviderId();
providerIdPropertyName = NiFiProperties.STATE_MANAGEMENT_CLUSTER_PROVIDER_ID;
providerDescription = "Cluster State Provider";
providerXmlElementName = "cluster-provider";
oppositeScopeXmlElementName = "local-provider";
break;
case LOCAL:
providerId = properties.getLocalStateProviderId();
providerIdPropertyName = NiFiProperties.STATE_MANAGEMENT_LOCAL_PROVIDER_ID;
providerDescription = "Local State Provider";
providerXmlElementName = "local-provider";
oppositeScopeXmlElementName = "cluster-provider";
break;
default:
throw new AssertionError("Attempted to create State Provider for unknown Scope: " + scope);
}
if (!configFile.exists()) {
throw new IllegalStateException("Cannot create " + providerDescription + " because the State Management Configuration File " + configFile + " does not exist");
}
if (!configFile.canRead()) {
throw new IllegalStateException("Cannot create " + providerDescription + " because the State Management Configuration File " + configFile + " cannot be read");
}
if (providerId == null) {
if (scope == StateProviderScope.CLUSTER) {
throw new IllegalStateException("Cannot create Cluster State Provider because the '" + providerIdPropertyName
+ "' property is missing from the NiFi Properties file. In order to run NiFi in a cluster, the " + providerIdPropertyName
+ " property must be configured in nifi.properties");
}
throw new IllegalStateException("Cannot create " + providerDescription + " because the '" + providerIdPropertyName
+ "' property is missing from the NiFi Properties file");
}
if (providerId.trim().isEmpty()) {
throw new IllegalStateException("Cannot create " + providerDescription + " because the '" + providerIdPropertyName
+ "' property in the NiFi Properties file has no value set. This is a required property and must reference the identifier of one of the "
+ providerXmlElementName + " elements in the State Management Configuraiton File (" + configFile + ")");
}
final StateManagerConfiguration config = StateManagerConfiguration.parse(configFile);
final StateProviderConfiguration providerConfig = config.getStateProviderConfiguration(providerId);
if (providerConfig == null) {
throw new IllegalStateException("Cannot create " + providerDescription + " because the '" + providerIdPropertyName
+ "' property in the NiFi Properties file is set to '" + providerId + "', but there is no " + providerXmlElementName
+ " entry in the State Management Configuration File (" + configFile + ") with this id");
}
if (providerConfig.getScope() != scope) {
throw new IllegalStateException("Cannot create " + providerDescription + " because the '" + providerIdPropertyName
+ "' property in the NiFi Properties file is set to '" + providerId + "', but this id is assigned to a " + oppositeScopeXmlElementName
+ " entry in the State Management Configuration File (" + configFile + "), rather than a " + providerXmlElementName + " entry");
}
final String providerClassName = providerConfig.getClassName();
final StateProvider provider;
try {
provider = instantiateStateProvider(providerClassName);
} catch (final Exception e) {
throw new RuntimeException("Cannot create " + providerDescription + " of type " + providerClassName, e);
}
final Map<PropertyDescriptor, PropertyValue> propertyMap = new HashMap<>();
final Map<PropertyDescriptor, String> propertyStringMap = new HashMap<>();
for (final PropertyDescriptor descriptor : provider.getPropertyDescriptors()) {
propertyMap.put(descriptor, new StandardPropertyValue(descriptor.getDefaultValue(), null));
propertyStringMap.put(descriptor, descriptor.getDefaultValue());
}
for (final Map.Entry<String, String> entry : providerConfig.getProperties().entrySet()) {
final PropertyDescriptor descriptor = provider.getPropertyDescriptor(entry.getKey());
propertyStringMap.put(descriptor, entry.getValue());
propertyMap.put(descriptor, new StandardPropertyValue(entry.getValue(), null));
}
final SSLContext sslContext = SslContextFactory.createSslContext(properties, false);
final StateProviderInitializationContext initContext = new StandardStateProviderInitializationContext(providerId, propertyMap, sslContext);
synchronized (provider) {
provider.initialize(initContext);
}
final ValidationContext validationContext = new StandardValidationContext(null, propertyStringMap, null);
final Collection<ValidationResult> results = provider.validate(validationContext);
final StringBuilder validationFailures = new StringBuilder();
int invalidCount = 0;
for (final ValidationResult result : results) {
if (!result.isValid()) {
validationFailures.append(result.toString()).append("\n");
invalidCount++;
}
}
if (invalidCount > 0) {
throw new IllegalStateException("Could not initialize State Providers because the " + providerDescription + " is not valid. The following "
+ invalidCount + " Validation Errors occurred:\n" + validationFailures.toString() + "\nPlease check the configuration of the " + providerDescription + " with ID ["
+ providerId.trim() + "] in the file " + configFile.getAbsolutePath());
}
return provider;
}
private static StateProvider instantiateStateProvider(final String type) throws ClassNotFoundException, InstantiationException, IllegalAccessException {
final ClassLoader ctxClassLoader = Thread.currentThread().getContextClassLoader();
try {
final ClassLoader detectedClassLoaderForType = ExtensionManager.getClassLoader(type);
final Class<?> rawClass;
if (detectedClassLoaderForType == null) {
// try to find from the current class loader
rawClass = Class.forName(type);
} else {
// try to find from the registered classloader for that type
rawClass = Class.forName(type, true, ExtensionManager.getClassLoader(type));
}
Thread.currentThread().setContextClassLoader(detectedClassLoaderForType);
final Class<? extends StateProvider> mgrClass = rawClass.asSubclass(StateProvider.class);
return mgrClass.newInstance();
} finally {
if (ctxClassLoader != null) {
Thread.currentThread().setContextClassLoader(ctxClassLoader);
}
}
}
/**
* Returns the State Manager that has been created for the given component ID, or <code>null</code> if none exists
*
* @return the StateManager that can be used by the component with the given ID, or <code>null</code> if none exists
*/
@Override
public synchronized StateManager getStateManager(final String componentId) {
StateManager stateManager = stateManagers.get(componentId);
if (stateManager != null) {
return stateManager;
}
stateManager = new StandardStateManager(localStateProvider, clusterStateProvider, componentId);
stateManagers.put(componentId, stateManager);
return stateManager;
}
@Override
public synchronized void shutdown() {
localStateProvider.shutdown();
if (clusterStateProvider != null) {
clusterStateProvider.shutdown();
}
}
@Override
public void enableClusterProvider() {
clusterStateProvider.enable();
}
@Override
public void disableClusterProvider() {
clusterStateProvider.disable();
}
@Override
public void onComponentRemoved(final String componentId) {
final StateManager mgr = stateManagers.remove(componentId);
if (mgr == null) {
return;
}
try {
mgr.clear(Scope.CLUSTER);
} catch (final Exception e) {
logger.warn("Component with ID {} was removed from NiFi instance but failed to clear clustered state for the component", e);
}
try {
mgr.clear(Scope.LOCAL);
} catch (final Exception e) {
logger.warn("Component with ID {} was removed from NiFi instance but failed to clear local state for the component", e);
}
try {
localStateProvider.onComponentRemoved(componentId);
} catch (final Exception e) {
logger.warn("Component with ID {} was removed from NiFi instance but failed to cleanup resources used to maintain its local state", e);
}
if (clusterStateProvider != null) {
try {
clusterStateProvider.onComponentRemoved(componentId);
} catch (final Exception e) {
logger.warn("Component with ID {} was removed from NiFi instance but failed to cleanup resources used to maintain its clustered state", e);
}
}
}
}

View File

@ -0,0 +1,58 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.state.providers;
import java.io.IOException;
import org.apache.nifi.components.AbstractConfigurableComponent;
import org.apache.nifi.components.state.StateProvider;
import org.apache.nifi.components.state.StateProviderInitializationContext;
public abstract class AbstractStateProvider extends AbstractConfigurableComponent implements StateProvider {
private String identifier;
private volatile boolean enabled;
@Override
public final void initialize(final StateProviderInitializationContext context) throws IOException {
this.identifier = context.getIdentifier();
init(context);
}
@Override
public String getIdentifier() {
return identifier;
}
@Override
public void enable() {
enabled = true;
}
@Override
public void disable() {
enabled = false;
}
@Override
public boolean isEnabled() {
return enabled;
}
public abstract void init(final StateProviderInitializationContext context) throws IOException;
}

View File

@ -0,0 +1,224 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.state.providers.local;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.state.StateMap;
import org.apache.nifi.components.state.StateProviderInitializationContext;
import org.apache.nifi.controller.state.StandardStateMap;
import org.apache.nifi.controller.state.StateMapSerDe;
import org.apache.nifi.controller.state.StateMapUpdate;
import org.apache.nifi.controller.state.providers.AbstractStateProvider;
import org.apache.nifi.processor.util.StandardValidators;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.wali.MinimalLockingWriteAheadLog;
import org.wali.UpdateType;
import org.wali.WriteAheadRepository;
/**
* Provides state management for local (node-only) state, backed by a write-ahead log
*/
public class WriteAheadLocalStateProvider extends AbstractStateProvider {
private static final Logger logger = LoggerFactory.getLogger(WriteAheadLocalStateProvider.class);
// TODO: CREATE BACKGROUND THREAD OR USE EXECUTOR (in StateProviderInitializationContext?) to schedule checkpointing.
private static final long CHECKPOINT_NANOS = TimeUnit.MINUTES.toNanos(2);
private final StateMapSerDe serde;
private final ConcurrentMap<String, ComponentProvider> componentProviders = new ConcurrentHashMap<>();
static final PropertyDescriptor PATH = new PropertyDescriptor.Builder()
.name("Directory")
.description("The directory where the Provider should store its data")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.defaultValue("./state")
.required(true)
.build();
private WriteAheadRepository<StateMapUpdate> writeAheadLog;
private AtomicLong versionGenerator;
public WriteAheadLocalStateProvider() {
serde = new StateMapSerDe();
}
@Override
public synchronized void init(final StateProviderInitializationContext context) throws IOException {
final File basePath = new File(context.getProperty(PATH).getValue());
if (!basePath.exists() && !basePath.mkdirs()) {
throw new RuntimeException("Cannot Initialize Local State Provider because the 'Directory' property is set to \"" + basePath + "\", but that directory could not be created");
}
if (!basePath.isDirectory()) {
throw new RuntimeException("Cannot Initialize Local State Provider because the 'Directory' property is set to \"" + basePath + "\", but that is a file, rather than a directory");
}
if (!basePath.canWrite()) {
throw new RuntimeException("Cannot Initialize Local State Provider because the 'Directory' property is set to \"" + basePath + "\", but that directory cannot be written to");
}
if (!basePath.canRead()) {
throw new RuntimeException("Cannot Initialize Local State Provider because the 'Directory' property is set to \"" + basePath + "\", but that directory cannot be read");
}
versionGenerator = new AtomicLong(-1L);
writeAheadLog = new MinimalLockingWriteAheadLog<>(basePath.toPath(), 16, serde, null);
final Collection<StateMapUpdate> updates = writeAheadLog.recoverRecords();
long maxRecordVersion = -1L;
for (final StateMapUpdate update : updates) {
if (update.getUpdateType() == UpdateType.DELETE) {
continue;
}
final long recordVersion = update.getStateMap().getVersion();
if (recordVersion > maxRecordVersion) {
maxRecordVersion = recordVersion;
}
final String componentId = update.getComponentId();
componentProviders.put(componentId, new ComponentProvider(writeAheadLog, versionGenerator, componentId, update.getStateMap()));
}
// keep a separate maxRecordVersion and set it at the end so that we don't have to continually update an AtomicLong, which is more
// expensive than just keeping track of a local 'long' variable. Since we won't actually increment this at any point until this after
// the init() method completes, this is okay to do.
versionGenerator.set(maxRecordVersion);
}
@Override
public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
final List<PropertyDescriptor> properties = new ArrayList<>();
properties.add(PATH);
return properties;
}
@Override
public synchronized void shutdown() {
try {
writeAheadLog.shutdown();
} catch (final IOException ioe) {
logger.warn("Failed to shut down {} successfully due to {}", this, ioe.toString());
logger.warn("", ioe);
}
}
private ComponentProvider getProvider(final String componentId) {
ComponentProvider componentProvider = componentProviders.get(componentId);
if (componentProvider == null) {
final StateMap stateMap = new StandardStateMap(Collections.<String, String> emptyMap(), -1L);
componentProvider = new ComponentProvider(writeAheadLog, versionGenerator, componentId, stateMap);
final ComponentProvider existingComponentProvider = componentProviders.putIfAbsent(componentId, componentProvider);
if (existingComponentProvider != null) {
componentProvider = existingComponentProvider;
}
}
return componentProvider;
}
@Override
public StateMap getState(final String componentId) throws IOException {
return getProvider(componentId).getState();
}
@Override
public void setState(final Map<String, String> state, final String componentId) throws IOException {
getProvider(componentId).setState(state);
}
@Override
public boolean replace(final StateMap oldValue, final Map<String, String> newValue, final String componentId) throws IOException {
return getProvider(componentId).replace(oldValue, newValue);
}
@Override
public void clear(final String componentId) throws IOException {
getProvider(componentId).clear();
}
@Override
public void onComponentRemoved(final String componentId) throws IOException {
clear(componentId);
componentProviders.remove(componentId);
}
private static class ComponentProvider {
private final AtomicLong versionGenerator;
private final WriteAheadRepository<StateMapUpdate> wal;
private final String componentId;
private StateMap stateMap;
public ComponentProvider(final WriteAheadRepository<StateMapUpdate> wal, final AtomicLong versionGenerator, final String componentId, final StateMap stateMap) {
this.wal = wal;
this.versionGenerator = versionGenerator;
this.componentId = componentId;
this.stateMap = stateMap;
}
public synchronized StateMap getState() throws IOException {
return stateMap;
}
// synchronized because we need to ensure that update of state in WAL and updating of local stateMap variable is atomic.
// Additionally, the implementation of WriteAheadRepository that we are using requires that only a single thread update the
// repository at a time for a record with the same key. I.e., many threads can update the repository at once, as long as they
// are not updating the repository with records that have the same identifier.
public synchronized void setState(final Map<String, String> state) throws IOException {
stateMap = new StandardStateMap(state, versionGenerator.incrementAndGet());
final StateMapUpdate updateRecord = new StateMapUpdate(stateMap, componentId, UpdateType.UPDATE);
wal.update(Collections.singleton(updateRecord), false);
}
// see above explanation as to why this method is synchronized.
public synchronized boolean replace(final StateMap oldValue, final Map<String, String> newValue) throws IOException {
if (stateMap != oldValue) {
return false;
}
stateMap = new StandardStateMap(new HashMap<>(newValue), versionGenerator.incrementAndGet());
final StateMapUpdate updateRecord = new StateMapUpdate(stateMap, componentId, UpdateType.UPDATE);
wal.update(Collections.singleton(updateRecord), false);
return true;
}
public synchronized void clear() throws IOException {
stateMap = new StandardStateMap(null, versionGenerator.incrementAndGet());
final StateMapUpdate update = new StateMapUpdate(stateMap, componentId, UpdateType.UPDATE);
wal.update(Collections.singleton(update), false);
}
}
}

View File

@ -0,0 +1,330 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.state.providers.zookeeper;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.state.StateMap;
import org.apache.nifi.components.state.StateProviderInitializationContext;
import org.apache.nifi.controller.state.StandardStateMap;
import org.apache.nifi.controller.state.providers.AbstractStateProvider;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.Code;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.ZKUtil;
import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Stat;
public class ZooKeeperStateProvider extends AbstractStateProvider {
static final PropertyDescriptor CONNECTION_STRING = new PropertyDescriptor.Builder()
.name("Connect String")
.description("The ZooKeeper Connect String to use. This is a comma-separated list of hostnames/IP addresses, such as \"host1, host2, 127.0.0.1, host4, host5\"")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.required(false)
.build();
static final PropertyDescriptor SESSION_TIMEOUT = new PropertyDescriptor.Builder()
.name("Session Timeout")
.description("Specifies how long this instance of NiFi is allowed to be disconnected from ZooKeeper before creating a new ZooKeeper Session")
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.defaultValue("30 sec")
.required(true)
.build();
static final PropertyDescriptor ROOT_NODE = new PropertyDescriptor.Builder()
.name("Root Node")
.description("The Root Node to use in ZooKeeper to store state in")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.defaultValue("/nifi")
.required(true)
.build();
private final List<ACL> acl;
private ZooKeeper zooKeeper;
private int timeoutMillis;
private String rootNode;
private String connectionString;
private static final int ENCODING_VERSION = 1;
public ZooKeeperStateProvider() throws Exception {
// TODO: Provide SSL Context
// TODO: Use more appropriate acl
acl = Ids.OPEN_ACL_UNSAFE;
}
@Override
public synchronized void init(final StateProviderInitializationContext context) {
connectionString = context.getProperty(CONNECTION_STRING).getValue();
rootNode = context.getProperty(ROOT_NODE).getValue();
timeoutMillis = context.getProperty(SESSION_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).intValue();
}
@Override
public synchronized void shutdown() {
if (zooKeeper != null) {
try {
zooKeeper.close();
} catch (final InterruptedException e) {
Thread.currentThread().interrupt();
}
}
zooKeeper = null;
}
@Override
public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
final List<PropertyDescriptor> properties = new ArrayList<>();
properties.add(CONNECTION_STRING);
properties.add(SESSION_TIMEOUT);
properties.add(ROOT_NODE);
return properties;
}
private synchronized ZooKeeper getZooKeeper() throws IOException {
if (zooKeeper != null && !zooKeeper.getState().isAlive()) {
invalidateClient();
}
if (zooKeeper == null) {
zooKeeper = new ZooKeeper(connectionString, timeoutMillis, new Watcher() {
@Override
public void process(WatchedEvent event) {
}
});
}
return zooKeeper;
}
private synchronized void invalidateClient() {
shutdown();
}
private String getComponentPath(final String componentId) {
return rootNode + "/components/" + componentId;
}
private void verifyEnabled() throws IOException {
if (!isEnabled()) {
throw new IOException("Cannot update or retrieve cluster state becuase node is no longer connected to a cluster");
}
}
@Override
public void onComponentRemoved(final String componentId) throws IOException {
try {
ZKUtil.deleteRecursive(getZooKeeper(), getComponentPath(componentId));
} catch (final KeeperException ke) {
// Node doesn't exist so just ignore
if (Code.NONODE == ke.code()) {
return;
}
if (Code.SESSIONEXPIRED == ke.code()) {
invalidateClient();
onComponentRemoved(componentId);
}
throw new IOException("Unable to remove state for component with ID '" + componentId + "' from ZooKeeper", ke);
} catch (final InterruptedException e) {
Thread.currentThread().interrupt();
throw new IOException("Failed to remove state for component with ID '" + componentId + "' from ZooKeeper due to being interrupted", e);
}
}
@Override
public void setState(final Map<String, String> state, final String componentId) throws IOException {
setState(state, -1, componentId);
}
private byte[] serialize(final Map<String, String> stateValues) throws IOException {
try (final ByteArrayOutputStream baos = new ByteArrayOutputStream();
final DataOutputStream dos = new DataOutputStream(baos)) {
dos.writeInt(ENCODING_VERSION);
dos.writeInt(stateValues.size());
for (final Map.Entry<String, String> entry : stateValues.entrySet()) {
dos.writeUTF(entry.getKey());
dos.writeUTF(entry.getValue());
}
return baos.toByteArray();
}
}
private StateMap deserialize(final byte[] data, final int recordVersion, final String componentId) throws IOException {
try (final ByteArrayInputStream bais = new ByteArrayInputStream(data);
final DataInputStream dis = new DataInputStream(bais)) {
final int encodingVersion = dis.readInt();
if (encodingVersion > ENCODING_VERSION) {
throw new IOException("Retrieved a response from ZooKeeper when retrieving state for component with ID " + componentId
+ ", but the response was encoded using the ZooKeeperStateProvider Encoding Version of " + encodingVersion
+ " but this instance can only decode versions up to " + ENCODING_VERSION
+ "; it appears that the state was encoded using a newer version of NiFi than is currently running. This information cannot be decoded.");
}
final int numEntries = dis.readInt();
final Map<String, String> stateValues = new HashMap<>(numEntries);
for (int i = 0; i < numEntries; i++) {
final String key = dis.readUTF();
final String value = dis.readUTF();
stateValues.put(key, value);
}
return new StandardStateMap(stateValues, recordVersion);
}
}
private void setState(final Map<String, String> stateValues, final int version, final String componentId) throws IOException {
verifyEnabled();
try {
final String path = getComponentPath(componentId);
final byte[] data = serialize(stateValues);
final ZooKeeper keeper = getZooKeeper();
try {
keeper.setData(path, data, version);
} catch (final KeeperException ke) {
if (ke.code() == Code.NONODE) {
createNode(path, data);
} else {
throw ke;
}
}
} catch (final InterruptedException e) {
Thread.currentThread().interrupt();
throw new IOException("Failed to set cluster-wide state in ZooKeeper for component with ID " + componentId + " due to interruption", e);
} catch (final KeeperException ke) {
if (Code.SESSIONEXPIRED == ke.code()) {
invalidateClient();
setState(stateValues, version, componentId);
}
throw new IOException("Failed to set cluster-wide state in ZooKeeper for component with ID " + componentId, ke);
} catch (final IOException ioe) {
throw new IOException("Failed to set cluster-wide state in ZooKeeper for component with ID " + componentId, ioe);
}
}
private void createNode(final String path, final byte[] data) throws IOException, KeeperException {
try {
getZooKeeper().create(path, data, acl, CreateMode.PERSISTENT);
} catch (final InterruptedException ie) {
throw new IOException("Failed to update cluster-wide state due to interruption", ie);
} catch (final KeeperException ke) {
if (ke.code() == Code.NONODE) {
final String parentPath = StringUtils.substringBeforeLast(path, "/");
createNode(parentPath, null);
createNode(path, data);
return;
}
if (Code.SESSIONEXPIRED == ke.code()) {
invalidateClient();
createNode(path, data);
}
// Node already exists. Node must have been created by "someone else". Just set the data.
if (ke.code() == Code.NODEEXISTS) {
try {
getZooKeeper().setData(path, data, -1);
} catch (final KeeperException ke1) {
// Node no longer exists -- it was removed by someone else. Go recreate the node.
if (ke1.code() == Code.NONODE) {
createNode(path, data);
}
} catch (final InterruptedException ie) {
throw new IOException("Failed to update cluster-wide state due to interruption", ie);
}
}
throw ke;
}
}
@Override
public StateMap getState(final String componentId) throws IOException {
verifyEnabled();
try {
final Stat stat = new Stat();
final String path = getComponentPath(componentId);
final byte[] data = getZooKeeper().getData(path, false, stat);
final StateMap stateMap = deserialize(data, stat.getVersion(), componentId);
return stateMap;
} catch (final InterruptedException e) {
Thread.currentThread().interrupt();
throw new IOException("Failed to obtain value from ZooKeeper for component with ID " + componentId + ", due to interruption", e);
} catch (final KeeperException ke) {
if (ke.code() == Code.NONODE) {
return new StandardStateMap(null, -1L);
}
if (Code.SESSIONEXPIRED == ke.code()) {
invalidateClient();
return getState(componentId);
}
throw new IOException("Failed to obtain value from ZooKeeper for component with ID " + componentId, ke);
} catch (final IOException ioe) {
// provide more context in the error message
throw new IOException("Failed to obtain value from ZooKeeper for component with ID " + componentId, ioe);
}
}
@Override
public boolean replace(final StateMap oldValue, final Map<String, String> newValue, final String componentId) throws IOException {
verifyEnabled();
try {
setState(newValue, (int) oldValue.getVersion(), componentId);
return true;
} catch (final IOException ioe) {
return false;
}
}
@Override
public void clear(final String componentId) throws IOException {
verifyEnabled();
setState(Collections.<String, String> emptyMap(), componentId);
}
}

View File

@ -0,0 +1,77 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.state.server;
import java.io.BufferedInputStream;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.util.Properties;
import org.apache.nifi.util.NiFiProperties;
import org.apache.zookeeper.server.ServerConfig;
import org.apache.zookeeper.server.ZooKeeperServerMain;
import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class ZooKeeperStateServer extends ZooKeeperServerMain {
private static final Logger logger = LoggerFactory.getLogger(ZooKeeperStateServer.class);
private QuorumPeerConfig quorumPeerConfig;
private ZooKeeperStateServer(final Properties zkProperties) throws IOException, ConfigException {
quorumPeerConfig = new QuorumPeerConfig();
quorumPeerConfig.parseProperties(zkProperties);
}
public synchronized void start() throws IOException {
logger.info("Starting Embedded ZooKeeper Server");
final ServerConfig serverConfig = new ServerConfig();
serverConfig.readFrom(quorumPeerConfig);
runFromConfig(serverConfig);
}
@Override
public synchronized void shutdown() {
super.shutdown();
}
public static ZooKeeperStateServer create(final NiFiProperties properties) throws IOException, ConfigException {
final File propsFile = properties.getEmbeddedZooKeeperPropertiesFile();
if (propsFile == null) {
return null;
}
if (!propsFile.exists() || !propsFile.canRead()) {
throw new IOException("Cannot create Embedded ZooKeeper Server because the Properties File " + propsFile.getAbsolutePath()
+ " referenced in nifi.properties does not exist or cannot be read");
}
final Properties zkProperties = new Properties();
try (final InputStream fis = new FileInputStream(propsFile);
final InputStream bis = new BufferedInputStream(fis)) {
zkProperties.load(bis);
}
return new ZooKeeperStateServer(zkProperties);
}
}

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