From 57dadb7286c49d035df83ac565b0d72817469ba4 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Mon, 11 Jan 2016 08:28:12 -0500 Subject: [PATCH] NIFI-259: Initial implementation of State Management feature --- .../apache/nifi/components/state/Scope.java | 36 ++ .../nifi/components/state/StateManager.java | 84 +++++ .../nifi/components/state/StateMap.java | 51 +++ .../nifi/components/state/StateProvider.java | 127 +++++++ .../StateProviderInitializationContext.java | 56 +++ ...ontrollerServiceInitializationContext.java | 6 + .../apache/nifi/processor/ProcessContext.java | 6 + .../apache/nifi/processor/ProcessSession.java | 1 + .../nifi/reporting/ReportingContext.java | 6 + nifi-assembly/pom.xml | 6 + .../org/apache/nifi/util/NiFiProperties.java | 34 ++ .../client/socket/EndpointConnectionPool.java | 2 +- .../nifi/remote/cluster/NodeInformation.java | 16 +- .../cluster/NodeInformationAdapter.java | 2 +- .../main/asciidoc/administration-guide.adoc | 90 +++++ .../src/main/asciidoc/developer-guide.adoc | 65 ++++ .../apache/nifi/state/MockStateManager.java | 172 +++++++++ .../org/apache/nifi/state/MockStateMap.java | 49 +++ ...ontrollerServiceInitializationContext.java | 17 +- .../apache/nifi/util/MockProcessContext.java | 23 +- .../nifi/util/MockReportingContext.java | 10 +- .../nifi/util/MockValidationContext.java | 10 +- .../util/StandardProcessorTestRunner.java | 17 +- .../java/org/apache/nifi/util/TestRunner.java | 6 + .../nifi/util/TestMockProcessContext.java | 2 +- ...ontrollerServiceInitializationContext.java | 8 +- .../mock/MockProcessContext.java | 8 +- .../nifi/cluster/protocol/ClusterNodes.java | 39 +++ .../cluster/protocol/ConnectionResponse.java | 27 +- .../nifi/cluster/protocol/NodeIdentifier.java | 43 ++- .../message/AdaptedConnectionResponse.java | 10 +- .../jaxb/message/AdaptedNodeIdentifier.java | 31 +- .../message/ConnectionResponseAdapter.java | 8 +- .../jaxb/message/NodeIdentifierAdapter.java | 6 +- .../ClusterManagerProtocolSenderImplTest.java | 6 +- .../impl/NodeProtocolSenderImplTest.java | 9 +- .../org/apache/nifi/cluster/event/Event.java | 1 + .../cluster/event/impl/EventManagerImpl.java | 1 + .../impl/FileBasedClusterNodeFirewall.java | 3 +- .../cluster/manager/HttpClusterManager.java | 16 +- .../manager/HttpRequestReplicator.java | 3 +- .../cluster/manager/HttpResponseMapper.java | 1 + .../nifi/cluster/manager/NodeResponse.java | 7 +- .../exception/ConflictingNodeIdException.java | 46 +++ .../manager/impl/ClusteredEventAccess.java | 8 +- .../impl/ClusteredReportingContext.java | 12 +- .../impl/HttpRequestReplicatorImpl.java | 19 +- .../manager/impl/HttpResponseMapperImpl.java | 1 + .../manager/impl/WebClusterManager.java | 74 ++-- .../org/apache/nifi/cluster/node/Node.java | 1 - ...agerProtocolServiceLocatorFactoryBean.java | 1 - ...leBasedClusterNodeFirewallFactoryBean.java | 1 + .../reporting/ClusteredReportingTaskNode.java | 7 +- .../event/impl/EventManagerImplTest.java | 14 +- .../FileBasedClusterNodeFirewallTest.java | 8 +- .../DataFlowManagementServiceImplTest.java | 27 +- .../impl/HttpRequestReplicatorImplTest.java | 58 +-- .../impl/HttpResponseMapperImplTest.java | 21 +- .../manager/impl/TestWebClusterManager.java | 2 - .../manager/testutils/HttpRequest.java | 2 + .../manager/testutils/HttpResponse.java | 1 + .../cluster/manager/testutils/HttpServer.java | 1 + .../state/StateManagerProvider.java | 60 ++++ .../AbstractConfiguredComponent.java | 20 +- .../nifi/controller/ConfiguredComponent.java | 14 +- .../nifi-framework-core/pom.xml | 48 ++- .../apache/nifi/cluster/HeartbeatPayload.java | 18 - .../nifi/controller/FlowController.java | 136 +++++--- .../nifi/controller/StandardFlowService.java | 50 ++- .../controller/StandardFlowSynchronizer.java | 16 +- .../reporting/AbstractReportingTaskNode.java | 8 +- .../reporting/StandardReportingContext.java | 10 +- .../reporting/StandardReportingTaskNode.java | 2 +- .../repository/BatchingSessionFactory.java | 4 +- .../scheduling/ConnectableProcessContext.java | 10 +- .../EventDrivenSchedulingAgent.java | 21 +- .../scheduling/QuartzSchedulingAgent.java | 10 +- .../controller/scheduling/ScheduleState.java | 10 +- .../scheduling/StandardProcessScheduler.java | 31 +- .../TimerDrivenSchedulingAgent.java | 9 +- .../service/ControllerServiceLoader.java | 4 +- ...ontrollerServiceInitializationContext.java | 10 +- .../StandardControllerServiceNode.java | 8 +- .../StandardControllerServiceProvider.java | 16 +- .../nifi/controller/state/ClusterState.java | 38 ++ .../state/ConfigParseException.java | 30 ++ .../controller/state/NodeDescription.java | 39 +++ .../state/StandardStateManager.java | 73 ++++ .../controller/state/StandardStateMap.java | 49 +++ ...ardStateProviderInitializationContext.java | 60 ++++ .../nifi/controller/state/StateMapSerDe.java | 103 ++++++ .../nifi/controller/state/StateMapUpdate.java | 45 +++ .../state/StateProviderException.java | 30 ++ .../config/StateManagerConfiguration.java | 142 ++++++++ .../config/StateProviderConfiguration.java | 51 +++ .../state/config/StateProviderScope.java | 30 ++ .../manager/StandardStateManagerProvider.java | 295 ++++++++++++++++ .../providers/AbstractStateProvider.java | 58 +++ .../local/WriteAheadLocalStateProvider.java | 224 ++++++++++++ .../zookeeper/ZooKeeperStateProvider.java | 330 ++++++++++++++++++ .../state/server/ZooKeeperStateServer.java | 77 ++++ .../nifi/groups/StandardProcessGroup.java | 20 +- .../processor/StandardProcessContext.java | 10 +- .../processor/StandardSchedulingContext.java | 10 +- ...apache.nifi.components.state.StateProvider | 16 + .../TestStandardProcessScheduler.java | 24 +- ...StandardControllerServiceProviderTest.java | 28 +- ...TestStandardControllerServiceProvider.java | 78 +++-- .../providers/AbstractTestStateProvider.java | 153 ++++++++ .../TestWriteAheadLocalStateProvider.java | 90 +++++ .../zookeeper/TestZooKeeperStateProvider.java | 106 ++++++ .../src/main/resources/conf/nifi.properties | 14 + .../main/resources/conf/state-management.xml | 52 +++ .../main/resources/conf/zookeeper.properties | 30 ++ .../socket/ClusterManagerServerProtocol.java | 2 +- .../impl/StandardControllerServiceDAO.java | 4 +- .../web/dao/impl/StandardProcessorDAO.java | 4 +- .../dao/impl/StandardReportingTaskDAO.java | 4 +- .../resources/access-control/nifi.properties | 7 + .../access-control/state-management.xml | 38 ++ .../processors/hl7/ExtractHL7Attributes.java | 3 +- .../apache/nifi/processors/hl7/RouteHL7.java | 2 +- .../processors/kafka/test/EmbeddedKafka.java | 3 +- .../standard/AbstractListProcessor.java | 324 ++++++++--------- .../nifi/processors/standard/GetHTTP.java | 122 ++----- .../standard/TestAbstractListProcessor.java | 34 +- .../standard/TestDetectDuplicate.java | 42 +-- .../nifi/processors/standard/TestGetHTTP.java | 108 +----- .../standard/TestRouteOnAttribute.java | 5 +- pom.xml | 121 +++++-- 130 files changed, 4241 insertions(+), 827 deletions(-) create mode 100644 nifi-api/src/main/java/org/apache/nifi/components/state/Scope.java create mode 100644 nifi-api/src/main/java/org/apache/nifi/components/state/StateManager.java create mode 100644 nifi-api/src/main/java/org/apache/nifi/components/state/StateMap.java create mode 100644 nifi-api/src/main/java/org/apache/nifi/components/state/StateProvider.java create mode 100644 nifi-api/src/main/java/org/apache/nifi/components/state/StateProviderInitializationContext.java create mode 100644 nifi-mock/src/main/java/org/apache/nifi/state/MockStateManager.java create mode 100644 nifi-mock/src/main/java/org/apache/nifi/state/MockStateMap.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterNodes.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/ConflictingNodeIdException.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/state/StateManagerProvider.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/ClusterState.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/ConfigParseException.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/NodeDescription.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/StandardStateManager.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/StandardStateMap.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/StandardStateProviderInitializationContext.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/StateMapSerDe.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/StateMapUpdate.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/StateProviderException.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/config/StateManagerConfiguration.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/config/StateProviderConfiguration.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/config/StateProviderScope.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/manager/StandardStateManagerProvider.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/providers/AbstractStateProvider.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/providers/local/WriteAheadLocalStateProvider.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/providers/zookeeper/ZooKeeperStateProvider.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/server/ZooKeeperStateServer.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/META-INF/services/org.apache.nifi.components.state.StateProvider create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/state/providers/AbstractTestStateProvider.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/state/providers/local/TestWriteAheadLocalStateProvider.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/state/providers/zookeeper/TestZooKeeperStateProvider.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/state-management.xml create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/zookeeper.properties create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/access-control/state-management.xml diff --git a/nifi-api/src/main/java/org/apache/nifi/components/state/Scope.java b/nifi-api/src/main/java/org/apache/nifi/components/state/Scope.java new file mode 100644 index 0000000000..8daf12a546 --- /dev/null +++ b/nifi-api/src/main/java/org/apache/nifi/components/state/Scope.java @@ -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; +} diff --git a/nifi-api/src/main/java/org/apache/nifi/components/state/StateManager.java b/nifi-api/src/main/java/org/apache/nifi/components/state/StateManager.java new file mode 100644 index 0000000000..f52b2e3707 --- /dev/null +++ b/nifi-api/src/main/java/org/apache/nifi/components/state/StateManager.java @@ -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; + +/** + *

+ * The StateManager is responsible for providing NiFi components a mechanism for storing + * and retrieving state. + *

+ * + *

+ * 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." + *

+ */ +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 state, Scope scope) throws IOException; + + /** + * Returns the current state for the component. This return value will never be null. + * 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 true if the state was updated to the new value, false 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 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; +} diff --git a/nifi-api/src/main/java/org/apache/nifi/components/state/StateMap.java b/nifi-api/src/main/java/org/apache/nifi/components/state/StateMap.java new file mode 100644 index 0000000000..7984e42a59 --- /dev/null +++ b/nifi-api/src/main/java/org/apache/nifi/components/state/StateMap.java @@ -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 null 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 toMap(); +} diff --git a/nifi-api/src/main/java/org/apache/nifi/components/state/StateProvider.java b/nifi-api/src/main/java/org/apache/nifi/components/state/StateProvider.java new file mode 100644 index 0000000000..e0243f326d --- /dev/null +++ b/nifi-api/src/main/java/org/apache/nifi/components/state/StateProvider.java @@ -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; + +/** + *

+ * 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. + *

+ * + *

+ * 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. + *

+ * + * @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 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 true if the state was updated to the new value, false 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 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 true if the provider is enabled, false otherwise. + */ + boolean isEnabled(); +} diff --git a/nifi-api/src/main/java/org/apache/nifi/components/state/StateProviderInitializationContext.java b/nifi-api/src/main/java/org/apache/nifi/components/state/StateProviderInitializationContext.java new file mode 100644 index 0000000000..aaf5490d19 --- /dev/null +++ b/nifi-api/src/main/java/org/apache/nifi/components/state/StateProviderInitializationContext.java @@ -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 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 null if no SSLContext has been configured + */ + SSLContext getSSLContext(); +} diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/ControllerServiceInitializationContext.java b/nifi-api/src/main/java/org/apache/nifi/controller/ControllerServiceInitializationContext.java index 6fcee0c7bd..3486621c76 100644 --- a/nifi-api/src/main/java/org/apache/nifi/controller/ControllerServiceInitializationContext.java +++ b/nifi-api/src/main/java/org/apache/nifi/controller/ControllerServiceInitializationContext.java @@ -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(); } diff --git a/nifi-api/src/main/java/org/apache/nifi/processor/ProcessContext.java b/nifi-api/src/main/java/org/apache/nifi/processor/ProcessContext.java index cf1bb6cdfa..91ea1a2c34 100644 --- a/nifi-api/src/main/java/org/apache/nifi/processor/ProcessContext.java +++ b/nifi-api/src/main/java/org/apache/nifi/processor/ProcessContext.java @@ -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(); } diff --git a/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java b/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java index d0c5b46de6..5d867b7642 100644 --- a/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java +++ b/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java @@ -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; diff --git a/nifi-api/src/main/java/org/apache/nifi/reporting/ReportingContext.java b/nifi-api/src/main/java/org/apache/nifi/reporting/ReportingContext.java index 281194c7b5..cb131e2b51 100644 --- a/nifi-api/src/main/java/org/apache/nifi/reporting/ReportingContext.java +++ b/nifi-api/src/main/java/org/apache/nifi/reporting/ReportingContext.java @@ -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(); } diff --git a/nifi-assembly/pom.xml b/nifi-assembly/pom.xml index b8e83bd076..1b668a543c 100644 --- a/nifi-assembly/pom.xml +++ b/nifi-assembly/pom.xml @@ -283,6 +283,12 @@ language governing permissions and limitations under the License. --> ./conf/templates ./database_repository + ./conf/state-management.xml + false + ./conf/zookeeper.properties + local-provider + zk-provider + org.apache.nifi.controller.repository.WriteAheadFlowFileRepository ./flowfile_repository 256 diff --git a/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java b/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java index c82a220877..5b35328b4f 100644 --- a/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java +++ b/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java @@ -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)); + } } diff --git a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java index 18075dbbeb..81ca70da54 100644 --- a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java +++ b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java @@ -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; diff --git a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/NodeInformation.java b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/NodeInformation.java index 2041268693..abfcc85920 100644 --- a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/NodeInformation.java +++ b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/NodeInformation.java @@ -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 + "]"; } } diff --git a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/NodeInformationAdapter.java b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/NodeInformationAdapter.java index 440463cbd7..b2dead043c 100644 --- a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/NodeInformationAdapter.java +++ b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/NodeInformationAdapter.java @@ -30,7 +30,7 @@ public class NodeInformationAdapter extends XmlAdapter/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 :[:]. 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 <> 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 <> 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 <> 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. diff --git a/nifi-docs/src/main/asciidoc/developer-guide.adoc b/nifi-docs/src/main/asciidoc/developer-guide.adoc index aeea3dc0ba..f9bf675d00 100644 --- a/nifi-docs/src/main/asciidoc/developer-guide.adoc +++ b/nifi-docs/src/main/asciidoc/developer-guide.adoc @@ -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 +<> 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 . + +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 diff --git a/nifi-mock/src/main/java/org/apache/nifi/state/MockStateManager.java b/nifi-mock/src/main/java/org/apache/nifi/state/MockStateManager.java new file mode 100644 index 0000000000..c014ed069c --- /dev/null +++ b/nifi-mock/src/main/java/org/apache/nifi/state/MockStateManager.java @@ -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 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 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. 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 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 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()); + } +} diff --git a/nifi-mock/src/main/java/org/apache/nifi/state/MockStateMap.java b/nifi-mock/src/main/java/org/apache/nifi/state/MockStateMap.java new file mode 100644 index 0000000000..cfce4670a0 --- /dev/null +++ b/nifi-mock/src/main/java/org/apache/nifi/state/MockStateMap.java @@ -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 stateValues; + private final long version; + + public MockStateMap(final Map stateValues, final long version) { + this.stateValues = stateValues == null ? Collections. 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 toMap() { + return Collections.unmodifiableMap(stateValues); + } +} diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceInitializationContext.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceInitializationContext.java index bff1d628a4..6a6e4cf218 100644 --- a/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceInitializationContext.java +++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceInitializationContext.java @@ -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; + } } diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java index e8e4dd56b3..c641d24dbf 100644 --- a/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java +++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java @@ -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 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 connections = new HashSet<>(); private volatile Set 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 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 elRanges = Query.extractExpressionRanges(getProperty(property).getValue()); return (elRanges != null && !elRanges.isEmpty()); } + + @Override + public StateManager getStateManager() { + return stateManager; + } } diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockReportingContext.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockReportingContext.java index 63a987626c..33719ecd1c 100644 --- a/nifi-mock/src/main/java/org/apache/nifi/util/MockReportingContext.java +++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockReportingContext.java @@ -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 controllerServices; private final MockEventAccess eventAccess = new MockEventAccess(); private final Map properties = new HashMap<>(); + private final StateManager stateManager; private final Map> componentBulletinsCreated = new HashMap<>(); - public MockReportingContext(final Map controllerServices) { + public MockReportingContext(final Map controllerServices, final StateManager stateManager) { this.controllerServices = new HashMap<>(); + this.stateManager = stateManager; for (final Map.Entry 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; + } } diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java index d73a09bfe0..6442778c0d 100644 --- a/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java +++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java @@ -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 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 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; diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java b/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java index 925f0d8f2c..82206327b6 100644 --- a/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java +++ b/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java @@ -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 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 results = context.getControllerService(service.getIdentifier()).validate(validationContext); for (final ValidationResult result : results) { @@ -719,7 +722,7 @@ public class StandardProcessorTestRunner implements TestRunner { final Map curProps = configuration.getProperties(); final Map 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; + } } diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java b/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java index 6c8f192b35..378a92ee0a 100644 --- a/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java +++ b/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java @@ -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(); } diff --git a/nifi-mock/src/test/java/org/apache/nifi/util/TestMockProcessContext.java b/nifi-mock/src/test/java/org/apache/nifi/util/TestMockProcessContext.java index fbdea940ac..d48af63348 100644 --- a/nifi-mock/src/test/java/org/apache/nifi/util/TestMockProcessContext.java +++ b/nifi-mock/src/test/java/org/apache/nifi/util/TestMockProcessContext.java @@ -123,4 +123,4 @@ public class TestMockProcessContext { public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { } } -} +} \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceInitializationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceInitializationContext.java index 56216aa416..c1cca260c6 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceInitializationContext.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceInitializationContext.java @@ -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; + } + +} \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockProcessContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockProcessContext.java index edf04751da..1acdd49c18 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockProcessContext.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockProcessContext.java @@ -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; + } +} \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterNodes.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterNodes.java new file mode 100644 index 0000000000..ad31e68d5b --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterNodes.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.cluster.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 nodeIds; + + public ClusterNodes(final Collection nodeIds) { + this.nodeIds = nodeIds; + } + + public Collection getNodeIdentifiers() { + return Collections.unmodifiableCollection(nodeIds); + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionResponse.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionResponse.java index 4e06926bb9..96bde72ae8 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionResponse.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionResponse.java @@ -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; @@ -43,7 +43,7 @@ public class ConnectionResponse { private volatile String clusterManagerDN; public ConnectionResponse(final NodeIdentifier nodeIdentifier, final StandardDataFlow dataFlow, final boolean primary, - final Integer managerRemoteInputPort, final Boolean managerRemoteCommsSecure, final String instanceId) { + final Integer managerRemoteInputPort, final Boolean managerRemoteCommsSecure, final String instanceId) { if (nodeIdentifier == null) { throw new IllegalArgumentException("Node identifier may not be empty or null."); } else if (dataFlow == null) { @@ -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; } - } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeIdentifier.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeIdentifier.java index eff62b9039..f48012a9cd 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeIdentifier.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeIdentifier.java @@ -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; } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionResponse.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionResponse.java index 60467027db..4243b41465 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionResponse.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionResponse.java @@ -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() { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeIdentifier.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeIdentifier.java index 3bbf7b6fe8..beca0148ae 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeIdentifier.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeIdentifier.java @@ -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; + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionResponseAdapter.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionResponseAdapter.java index baabc333f8..b2c1c67ced 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionResponseAdapter.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionResponseAdapter.java @@ -30,7 +30,7 @@ public class ConnectionResponseAdapter extends XmlAdapter properties; private final Map preparedQueries; + private final StateManager stateManager; - public ClusteredReportingContext(final EventAccess eventAccess, final BulletinRepository bulletinRepository, - final Map properties, final ControllerServiceProvider serviceProvider) { + public ClusteredReportingContext(final EventAccess eventAccess, final BulletinRepository bulletinRepository, final Map 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 entry : properties.entrySet()) { @@ -206,4 +209,9 @@ public class ClusteredReportingContext implements ReportingContext { return null; } + + @Override + public StateManager getStateManager() { + return stateManager; + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImpl.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImpl.java index 32181e38a8..f86c290520 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImpl.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImpl.java @@ -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 HttpRequestReplicator interface. This implementation parallelizes the node HTTP requests using the given ExecutorService instance. Individual * requests may have connection and read timeouts set, which may be set during instance construction. Otherwise, the default is not to timeout. diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImpl.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImpl.java index c11df0591b..098573d8bf 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImpl.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImpl.java @@ -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; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java index 55e58acfcc..95cac50c12 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java @@ -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 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 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); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/node/Node.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/node/Node.java index 95da615092..3bb3c1ab78 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/node/Node.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/node/Node.java @@ -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; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/ClusterManagerProtocolServiceLocatorFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/ClusterManagerProtocolServiceLocatorFactoryBean.java index 2136dad1a0..c51e061215 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/ClusterManagerProtocolServiceLocatorFactoryBean.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/ClusterManagerProtocolServiceLocatorFactoryBean.java @@ -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; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/FileBasedClusterNodeFirewallFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/FileBasedClusterNodeFirewallFactoryBean.java index ef722986fb..9d064ebcc3 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/FileBasedClusterNodeFirewallFactoryBean.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/FileBasedClusterNodeFirewallFactoryBean.java @@ -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; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/controller/reporting/ClusteredReportingTaskNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/controller/reporting/ClusteredReportingTaskNode.java index 1ed5b30054..a23cfdd788 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/controller/reporting/ClusteredReportingTaskNode.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/controller/reporting/ClusteredReportingTaskNode.java @@ -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); } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/event/impl/EventManagerImplTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/event/impl/EventManagerImplTest.java index e823d274b7..6487a20d78 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/event/impl/EventManagerImplTest.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/event/impl/EventManagerImplTest.java @@ -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; /** diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/firewall/impl/FileBasedClusterNodeFirewallTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/firewall/impl/FileBasedClusterNodeFirewallTest.java index b5f76fb0d6..55c8768f4c 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/firewall/impl/FileBasedClusterNodeFirewallTest.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/firewall/impl/FileBasedClusterNodeFirewallTest.java @@ -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; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImplTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImplTest.java index b99cbeafe7..e526ea3301 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImplTest.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImplTest.java @@ -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 = ""; 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 = ""; 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 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 = ""; 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); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImplTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImplTest.java index b02eac0917..e3e9ac6156 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImplTest.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImplTest.java @@ -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 createNodes(int num, String host, int apiPort) { Set 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; } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImplTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImplTest.java index ebea63a300..ba6ca829da 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImplTest.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImplTest.java @@ -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"); } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/TestWebClusterManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/TestWebClusterManager.java index 13a192fdd4..55c6c31c8a 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/TestWebClusterManager.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/TestWebClusterManager.java @@ -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; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpRequest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpRequest.java index 4c3eeeebe1..89c9a0ab63 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpRequest.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpRequest.java @@ -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; /** diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpResponse.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpResponse.java index 7ae48061a9..26c892d649 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpResponse.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpResponse.java @@ -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; /** diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpServer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpServer.java index 3621475a65..e9cbd1d492 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpServer.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpServer.java @@ -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; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/state/StateManagerProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/state/StateManagerProvider.java new file mode 100644 index 0000000000..0b1ab8d52e --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/state/StateManagerProvider.java @@ -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; + +/** + *

+ * Interface that provides a mechanism for obtaining the {@link StateManager} for a particular component + *

+ */ +public interface StateManagerProvider { + /** + * Returns the StateManager for the component with the given ID, or null 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 null 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(); +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java index 978c612fbb..214467dc86 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java @@ -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,10 +114,12 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone } } - try { - component.onPropertyModified(descriptor, oldValue, value); - } catch (final Throwable t) { - // nothing really to do here... + if (triggerOnPropertyModified) { + try { + component.onPropertyModified(descriptor, oldValue, value); + } catch (final Exception e) { + // nothing really to do here... + } } } } @@ -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 } } - component.onPropertyModified(descriptor, value, null); + if (triggerOnPropertyModified) { + component.onPropertyModified(descriptor, value, null); + } + return true; } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.java index 8b2794d6bd..7e497002cc 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.java @@ -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 true, 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 true, 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 getProperties(); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml index 783235a317..882695e1ca 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml @@ -1,19 +1,16 @@ - - + + 4.0.0 org.apache.nifi @@ -119,6 +116,27 @@ org.apache.nifi nifi-write-ahead-log + + org.apache.zookeeper + zookeeper + + + + org.apache.nifi + nifi-processor-utils + + + + org.apache.curator + curator-test + test + + + org.testng + testng + test + + org.apache.nifi nifi-mock diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/cluster/HeartbeatPayload.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/cluster/HeartbeatPayload.java index 54142596fe..ff3ad4e8db 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/cluster/HeartbeatPayload.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/cluster/HeartbeatPayload.java @@ -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; } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java index dd3b6870d1..95c93ba831 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java @@ -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 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 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 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 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()); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java index 1511293c60..6250c5a86d 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java @@ -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(); @@ -507,7 +528,7 @@ public class StandardFlowService implements FlowService, ProtocolHandler { // reconnect final ConnectionResponse connectionResponse = new ConnectionResponse(nodeId, request.getDataFlow(), request.isPrimary(), - request.getManagerRemoteSiteListeningPort(), request.isManagerRemoteSiteCommsSecure(), request.getInstanceId()); + request.getManagerRemoteSiteListeningPort(), request.isManagerRemoteSiteCommsSecure(), request.getInstanceId()); connectionResponse.setClusterManagerDN(request.getRequestorDN()); loadFromConnectionResponse(connectionResponse); @@ -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 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 { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java index 0dd3b643ab..07dd58b2b9 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java @@ -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 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 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 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 taskElements = DomUtils.getChildElementsByTagName(reportingTasksElement, "reportingTask"); for (final Element taskElement : taskElements) { @@ -403,9 +403,9 @@ public class StandardFlowSynchronizer implements FlowSynchronizer { for (final Map.Entry 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 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); } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java index b537c30ec1..09d1b405fd 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java @@ -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(); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingContext.java index a4d337f622..11d1b51dec 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingContext.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingContext.java @@ -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 properties; private final Map preparedQueries; public StandardReportingContext(final FlowController flowController, final BulletinRepository bulletinRepository, - final Map properties, final ControllerServiceProvider serviceProvider) { + final Map 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 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()); + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingTaskNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingTaskNode.java index fe3af92fc1..1a40e8a02f 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingTaskNode.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingTaskNode.java @@ -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()); } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java index fd3f1cc59d..bac45afb55 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java @@ -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(); } - } - } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java index c68c78d05f..5e26c091fc 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java @@ -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; + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java index e5582eccb7..76d7c08320 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java @@ -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 connectionIndexMap = new ConcurrentHashMap<>(); private final ConcurrentMap scheduleStates = new ConcurrentHashMap<>(); - public EventDrivenSchedulingAgent(final FlowEngine flowEngine, final ControllerServiceProvider flowController, - final EventDrivenWorkerQueue workerQueue, final ProcessContextFactory contextFactory, final int maxThreadCount, final StringEncryptor encryptor) { + 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. diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/QuartzSchedulingAgent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/QuartzSchedulingAgent.java index 4278cee503..3fa6b5268b 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/QuartzSchedulingAgent.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/QuartzSchedulingAgent.java @@ -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 triggers = new ArrayList<>(); for (int i = 0; i < connectable.getMaxConcurrentTasks(); i++) { final Callable 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); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ScheduleState.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ScheduleState.java index ea0b4564f6..e03cc05abe 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ScheduleState.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ScheduleState.java @@ -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,12 +64,12 @@ 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 true 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 true and MUST call the methods annotated with + * {@link OnStopped @OnStopped} * * @return true if the caller is required to call Processor methods annotated with - * @OnStopped, false otherwise + * @OnStopped, false otherwise */ public boolean mustCallOnStoppedMethods() { return mustCallOnStoppedMethods.getAndSet(false); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java index 82fc812368..ee764e6371 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java @@ -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 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 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(); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java index 96cee20b1e..04db5498d1 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java @@ -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); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java index b5c3855138..f08d45bd5e 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java @@ -155,9 +155,9 @@ public class ControllerServiceLoader { for (final Map.Entry 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); } } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceInitializationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceInitializationContext.java index 02d6263051..482dabfd48 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceInitializationContext.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceInitializationContext.java @@ -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; + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java index ba03ee3ec0..9a4c1e6993 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java @@ -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(); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java index 6561eb8ee5..660f596d8c 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java @@ -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 controllerServices; private static final Set 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 diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/ClusterState.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/ClusterState.java new file mode 100644 index 0000000000..8227bbb667 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/ClusterState.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.controller.state; + +import java.util.Set; + +public interface ClusterState { + /** + * @return true if this instance of NiFi is connected to a cluster, false 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 null but will be empty if the node is not connected to a cluster + */ + Set getNodes(); +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/ConfigParseException.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/ConfigParseException.java new file mode 100644 index 0000000000..ae91fcf2ea --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/ConfigParseException.java @@ -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); + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/NodeDescription.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/NodeDescription.java new file mode 100644 index 0000000000..0c284bf59b --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/NodeDescription.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.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 null if the node is + * not running an embedded ZooKeeper server + */ + Integer getEmbeddedZooKeeperPort(); +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/StandardStateManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/StandardStateManager.java new file mode 100644 index 0000000000..e83bbac944 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/StandardStateManager.java @@ -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 newValue, final Scope scope) throws IOException { + return getProvider(scope).replace(oldValue, newValue, componentId); + } + + @Override + public void setState(final Map 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 + "]"; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/StandardStateMap.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/StandardStateMap.java new file mode 100644 index 0000000000..b006ac65b6 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/StandardStateMap.java @@ -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 stateValues; + private final long version; + + public StandardStateMap(final Map stateValues, final long version) { + this.stateValues = Collections.unmodifiableMap(stateValues == null ? Collections. 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 toMap() { + return stateValues; + } + +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/StandardStateProviderInitializationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/StandardStateProviderInitializationContext.java new file mode 100644 index 0000000000..c9e7b8ed2e --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/StandardStateProviderInitializationContext.java @@ -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 properties; + private final SSLContext sslContext; + + public StandardStateProviderInitializationContext(final String identifier, final Map properties, final SSLContext sslContext) { + this.id = identifier; + this.properties = new HashMap<>(properties); + this.sslContext = sslContext; + } + + @Override + public Map 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; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/StateMapSerDe.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/StateMapSerDe.java new file mode 100644 index 0000000000..5671f5a98c --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/StateMapSerDe.java @@ -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 { + 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 map = stateMap.toMap(); + out.writeInt(map.size()); + for (final Map.Entry entry : map.entrySet()) { + out.writeUTF(entry.getKey()); + out.writeUTF(entry.getValue()); + } + } + + @Override + public StateMapUpdate deserializeEdit(final DataInputStream in, final Map 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 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; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/StateMapUpdate.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/StateMapUpdate.java new file mode 100644 index 0000000000..9eb478bc91 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/StateMapUpdate.java @@ -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; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/StateProviderException.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/StateProviderException.java new file mode 100644 index 0000000000..f51d229684 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/StateProviderException.java @@ -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); + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/config/StateManagerConfiguration.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/config/StateManagerConfiguration.java new file mode 100644 index 0000000000..c8beceecd7 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/config/StateManagerConfiguration.java @@ -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 providers; + + private StateManagerConfiguration(final Map providerConfigs) { + this.providers = providerConfigs; + } + + public Map getStateProviderConfigurations() { + return Collections.unmodifiableMap(providers); + } + + public StateProviderConfiguration getStateProviderConfiguration(final String providerId) { + return providers.get(providerId); + } + + public List getStateProviderConfigurations(final StateProviderScope scope) { + final List 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 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 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 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 propertyElements = DomUtils.getChildElementsByTagName(providerElement, "property"); + final Map 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); + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/config/StateProviderConfiguration.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/config/StateProviderConfiguration.java new file mode 100644 index 0000000000..290a750c51 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/config/StateProviderConfiguration.java @@ -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 properties; + + public StateProviderConfiguration(final String id, final String className, final StateProviderScope scope, final Map 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 getProperties() { + return properties; + } + + public StateProviderScope getScope() { + return scope; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/config/StateProviderScope.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/config/StateProviderScope.java new file mode 100644 index 0000000000..40e1865910 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/config/StateProviderScope.java @@ -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; +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/manager/StandardStateManagerProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/manager/StandardStateManagerProvider.java new file mode 100644 index 0000000000..f88752778e --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/manager/StandardStateManagerProvider.java @@ -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 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 propertyMap = new HashMap<>(); + final Map 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 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 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 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 null if none exists + * + * @return the StateManager that can be used by the component with the given ID, or null 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); + } + } + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/providers/AbstractStateProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/providers/AbstractStateProvider.java new file mode 100644 index 0000000000..078dce32b0 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/providers/AbstractStateProvider.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.controller.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; +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/providers/local/WriteAheadLocalStateProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/providers/local/WriteAheadLocalStateProvider.java new file mode 100644 index 0000000000..c23e517e72 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/providers/local/WriteAheadLocalStateProvider.java @@ -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 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 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 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 getSupportedPropertyDescriptors() { + final List 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. 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 state, final String componentId) throws IOException { + getProvider(componentId).setState(state); + } + + @Override + public boolean replace(final StateMap oldValue, final Map 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 wal; + private final String componentId; + + private StateMap stateMap; + + public ComponentProvider(final WriteAheadRepository 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 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 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); + } + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/providers/zookeeper/ZooKeeperStateProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/providers/zookeeper/ZooKeeperStateProvider.java new file mode 100644 index 0000000000..f865c8bb62 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/providers/zookeeper/ZooKeeperStateProvider.java @@ -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; + + 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 getSupportedPropertyDescriptors() { + final List 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 state, final String componentId) throws IOException { + setState(state, -1, componentId); + } + + + private byte[] serialize(final Map 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 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 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 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 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. emptyMap(), componentId); + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/server/ZooKeeperStateServer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/server/ZooKeeperStateServer.java new file mode 100644 index 0000000000..8af9e5a23a --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/server/ZooKeeperStateServer.java @@ -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); + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java index 88fa43fd00..17f914903d 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java @@ -37,6 +37,7 @@ import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.nifi.annotation.lifecycle.OnRemoved; import org.apache.nifi.annotation.lifecycle.OnShutdown; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.state.StateManager; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.ConnectableType; import org.apache.nifi.connectable.Connection; @@ -44,6 +45,7 @@ import org.apache.nifi.connectable.Funnel; import org.apache.nifi.connectable.LocalPort; import org.apache.nifi.connectable.Port; import org.apache.nifi.connectable.Position; +import org.apache.nifi.controller.FlowController; import org.apache.nifi.controller.ProcessScheduler; import org.apache.nifi.controller.ProcessorNode; import org.apache.nifi.controller.ScheduledState; @@ -73,6 +75,7 @@ public final class StandardProcessGroup implements ProcessGroup { private final ProcessScheduler scheduler; private final ControllerServiceProvider controllerServiceProvider; + private final FlowController flowController; private final Map inputPorts = new HashMap<>(); private final Map outputPorts = new HashMap<>(); @@ -90,13 +93,16 @@ public final class StandardProcessGroup implements ProcessGroup { private static final Logger LOG = LoggerFactory.getLogger(StandardProcessGroup.class); - public StandardProcessGroup(final String id, final ControllerServiceProvider serviceProvider, final ProcessScheduler scheduler, final NiFiProperties nifiProps, final StringEncryptor encryptor) { + public StandardProcessGroup(final String id, final ControllerServiceProvider serviceProvider, final ProcessScheduler scheduler, final NiFiProperties nifiProps, final StringEncryptor encryptor, + final FlowController flowController) { this.id = id; this.controllerServiceProvider = serviceProvider; this.parent = new AtomicReference<>(); this.scheduler = scheduler; this.comments = new AtomicReference<>(""); this.encryptor = encryptor; + this.flowController = flowController; + name = new AtomicReference<>(); position = new AtomicReference<>(new Position(0D, 0D)); } @@ -327,11 +333,15 @@ public final class StandardProcessGroup implements ProcessGroup { } } + private StateManager getStateManager(final String componentId) { + return flowController.getStateManagerProvider().getStateManager(componentId); + } + @SuppressWarnings("deprecation") private void shutdown(final ProcessGroup procGroup) { for (final ProcessorNode node : procGroup.getProcessors()) { try (final NarCloseable x = NarCloseable.withNarLoader()) { - final StandardProcessContext processContext = new StandardProcessContext(node, controllerServiceProvider, encryptor); + final StandardProcessContext processContext = new StandardProcessContext(node, controllerServiceProvider, encryptor, getStateManager(node.getIdentifier())); ReflectionUtils.quietlyInvokeMethodsWithAnnotations(OnShutdown.class, org.apache.nifi.processor.annotation.OnShutdown.class, node.getProcessor(), processContext); } } @@ -681,8 +691,8 @@ public final class StandardProcessGroup implements ProcessGroup { } } - @SuppressWarnings("deprecation") @Override + @SuppressWarnings("deprecation") public void removeProcessor(final ProcessorNode processor) { final String id = requireNonNull(processor).getIdentifier(); writeLock.lock(); @@ -697,7 +707,7 @@ public final class StandardProcessGroup implements ProcessGroup { } try (final NarCloseable x = NarCloseable.withNarLoader()) { - final StandardProcessContext processContext = new StandardProcessContext(processor, controllerServiceProvider, encryptor); + final StandardProcessContext processContext = new StandardProcessContext(processor, controllerServiceProvider, encryptor, getStateManager(processor.getIdentifier())); ReflectionUtils.quietlyInvokeMethodsWithAnnotations(OnRemoved.class, org.apache.nifi.processor.annotation.OnRemoved.class, processor.getProcessor(), processContext); } catch (final Exception e) { throw new ComponentLifeCycleException("Failed to invoke 'OnRemoved' methods of " + processor, e); @@ -719,6 +729,8 @@ public final class StandardProcessGroup implements ProcessGroup { processors.remove(id); LogRepositoryFactory.getRepository(processor.getIdentifier()).removeAllObservers(); + flowController.getStateManagerProvider().onComponentRemoved(processor.getIdentifier()); + // must copy to avoid a concurrent modification final Set copy = new HashSet<>(processor.getConnections()); for (final Connection conn : copy) { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java index 8e52a46d95..5bb1a86bee 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java @@ -29,6 +29,7 @@ import org.apache.nifi.attribute.expression.language.StandardPropertyValue; import org.apache.nifi.attribute.expression.language.Query.Range; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyValue; +import org.apache.nifi.components.state.StateManager; import org.apache.nifi.connectable.Connection; import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.ControllerServiceLookup; @@ -43,11 +44,13 @@ public class StandardProcessContext implements ProcessContext, ControllerService private final ControllerServiceProvider controllerServiceProvider; private final Map preparedQueries; private final StringEncryptor encryptor; + private final StateManager stateManager; - public StandardProcessContext(final ProcessorNode processorNode, final ControllerServiceProvider controllerServiceProvider, final StringEncryptor encryptor) { + public StandardProcessContext(final ProcessorNode processorNode, final ControllerServiceProvider controllerServiceProvider, final StringEncryptor encryptor, final StateManager stateManager) { this.procNode = processorNode; this.controllerServiceProvider = controllerServiceProvider; this.encryptor = encryptor; + this.stateManager = stateManager; preparedQueries = new HashMap<>(); for (final Map.Entry entry : procNode.getProperties().entrySet()) { @@ -208,4 +211,9 @@ public class StandardProcessContext implements ProcessContext, ControllerService final List elRanges = Query.extractExpressionRanges(getProperty(property).getValue()); return (elRanges != null && !elRanges.isEmpty()); } + + @Override + public StateManager getStateManager() { + return stateManager; + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java index 8c903ee3d9..4985fe6707 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java @@ -24,6 +24,7 @@ import org.apache.nifi.attribute.expression.language.Query; import org.apache.nifi.attribute.expression.language.Query.Range; 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.controller.ProcessorNode; import org.apache.nifi.controller.service.ControllerServiceNode; @@ -35,11 +36,13 @@ public class StandardSchedulingContext implements SchedulingContext { private final ProcessContext processContext; private final ControllerServiceProvider serviceProvider; private final ProcessorNode processorNode; + private final StateManager stateManager; - public StandardSchedulingContext(final ProcessContext processContext, final ControllerServiceProvider serviceProvider, final ProcessorNode processorNode) { + public StandardSchedulingContext(final ProcessContext processContext, final ControllerServiceProvider serviceProvider, final ProcessorNode processorNode, final StateManager stateManager) { this.processContext = processContext; this.serviceProvider = serviceProvider; this.processorNode = processorNode; + this.stateManager = stateManager; } @Override @@ -139,4 +142,9 @@ public class StandardSchedulingContext implements SchedulingContext { final List elRanges = Query.extractExpressionRanges(getProperty(property).getValue()); return (elRanges != null && !elRanges.isEmpty()); } + + @Override + public StateManager getStateManager() { + return stateManager; + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/META-INF/services/org.apache.nifi.components.state.StateProvider b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/META-INF/services/org.apache.nifi.components.state.StateProvider new file mode 100644 index 0000000000..6786be9dda --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/META-INF/services/org.apache.nifi.components.state.StateProvider @@ -0,0 +1,16 @@ +# 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. +org.apache.nifi.controller.state.providers.local.WriteAheadLocalStateProvider +org.apache.nifi.components.state.providers.zookeeper.ZooKeeperStateProvider \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java index 770bb477d0..543bfd0f4c 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java @@ -38,6 +38,7 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.controller.AbstractControllerService; import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.components.state.StateManagerProvider; import org.apache.nifi.controller.Heartbeater; import org.apache.nifi.controller.ProcessScheduler; import org.apache.nifi.controller.ProcessorNode; @@ -72,12 +73,13 @@ public class TestStandardProcessScheduler { private StandardProcessScheduler scheduler = null; private ReportingTaskNode taskNode = null; private TestReportingTask reportingTask = null; + private StateManagerProvider stateMgrProvider = Mockito.mock(StateManagerProvider.class); @Before public void setup() throws InitializationException { System.setProperty("nifi.properties.file.path", "src/test/resources/nifi.properties"); this.refreshNiFiProperties(); - scheduler = new StandardProcessScheduler(Mockito.mock(Heartbeater.class), Mockito.mock(ControllerServiceProvider.class), null); + scheduler = new StandardProcessScheduler(Mockito.mock(Heartbeater.class), Mockito.mock(ControllerServiceProvider.class), null, stateMgrProvider); scheduler.setSchedulingAgent(SchedulingStrategy.TIMER_DRIVEN, Mockito.mock(SchedulingAgent.class)); reportingTask = new TestReportingTask(); @@ -117,12 +119,12 @@ public class TestStandardProcessScheduler { public void testDisableControllerServiceWithProcessorTryingToStartUsingIt() throws InterruptedException { final Processor proc = new ServiceReferencingProcessor(); - final ControllerServiceProvider serviceProvider = new StandardControllerServiceProvider(scheduler, null); + final ControllerServiceProvider serviceProvider = new StandardControllerServiceProvider(scheduler, null, Mockito.mock(StateManagerProvider.class)); final ControllerServiceNode service = serviceProvider.createControllerService(NoStartServiceImpl.class.getName(), "service", true); final ProcessorNode procNode = new StandardProcessorNode(proc, UUID.randomUUID().toString(), new StandardValidationContextFactory(serviceProvider), scheduler, serviceProvider); - procNode.setProperty(ServiceReferencingProcessor.SERVICE_DESC.getName(), service.getIdentifier()); + procNode.setProperty(ServiceReferencingProcessor.SERVICE_DESC.getName(), service.getIdentifier(), true); scheduler.enableControllerService(service); scheduler.startProcessor(procNode); @@ -198,7 +200,7 @@ public class TestStandardProcessScheduler { @Test public void validateServiceEnablementLogicHappensOnlyOnce() throws Exception { final ProcessScheduler scheduler = createScheduler(); - StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null); + StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null, stateMgrProvider); final ControllerServiceNode serviceNode = provider.createControllerService(SimpleTestService.class.getName(), "1", false); assertFalse(serviceNode.isActive()); @@ -237,7 +239,7 @@ public class TestStandardProcessScheduler { @Test public void validateDisabledServiceCantBeDisabled() throws Exception { final ProcessScheduler scheduler = createScheduler(); - StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null); + StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null, stateMgrProvider); final ControllerServiceNode serviceNode = provider.createControllerService(SimpleTestService.class.getName(), "1", false); SimpleTestService ts = (SimpleTestService) serviceNode.getControllerServiceImplementation(); @@ -275,7 +277,7 @@ public class TestStandardProcessScheduler { @Test public void validateEnabledServiceCanOnlyBeDisabledOnce() throws Exception { final ProcessScheduler scheduler = createScheduler(); - StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null); + StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null, stateMgrProvider); final ControllerServiceNode serviceNode = provider.createControllerService(SimpleTestService.class.getName(), "1", false); SimpleTestService ts = (SimpleTestService) serviceNode.getControllerServiceImplementation(); @@ -309,7 +311,7 @@ public class TestStandardProcessScheduler { @Test public void validateDisablingOfTheFailedService() throws Exception { final ProcessScheduler scheduler = createScheduler(); - StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null); + StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null, stateMgrProvider); final ControllerServiceNode serviceNode = provider.createControllerService(FailingService.class.getName(), "1", false); scheduler.enableControllerService(serviceNode); @@ -340,7 +342,7 @@ public class TestStandardProcessScheduler { @Test public void validateEnabledDisableMultiThread() throws Exception { final ProcessScheduler scheduler = createScheduler(); - StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null); + StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null, stateMgrProvider); ExecutorService executor = Executors.newCachedThreadPool(); for (int i = 0; i < 200; i++) { final ControllerServiceNode serviceNode = provider @@ -383,7 +385,7 @@ public class TestStandardProcessScheduler { @Test public void validateNeverEnablingServiceCanStillBeDisabled() throws Exception { final ProcessScheduler scheduler = createScheduler(); - StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null); + StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null, stateMgrProvider); final ControllerServiceNode serviceNode = provider.createControllerService(LongEnablingService.class.getName(), "1", false); LongEnablingService ts = (LongEnablingService) serviceNode.getControllerServiceImplementation(); @@ -408,7 +410,7 @@ public class TestStandardProcessScheduler { @Test public void validateLongEnablingServiceCanStillBeDisabled() throws Exception { final ProcessScheduler scheduler = createScheduler(); - StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null); + StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null, stateMgrProvider); final ControllerServiceNode serviceNode = provider.createControllerService(LongEnablingService.class.getName(), "1", false); LongEnablingService ts = (LongEnablingService) serviceNode.getControllerServiceImplementation(); @@ -505,6 +507,6 @@ public class TestStandardProcessScheduler { } private ProcessScheduler createScheduler() { - return new StandardProcessScheduler(mock(Heartbeater.class), null, null); + return new StandardProcessScheduler(mock(Heartbeater.class), null, null, stateMgrProvider); } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java index ed9af314bf..b9c0f7f07e 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java @@ -16,6 +16,8 @@ */ package org.apache.nifi.controller.service; +import org.apache.nifi.components.state.StateManager; +import org.apache.nifi.components.state.StateManagerProvider; import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.StandardFlowServiceTest; import org.apache.nifi.nar.ExtensionManager; @@ -25,6 +27,7 @@ import org.apache.nifi.util.NiFiProperties; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; +import org.mockito.Mockito; public class StandardControllerServiceProviderTest { @@ -43,7 +46,28 @@ public class StandardControllerServiceProviderTest { public void setup() throws Exception { String id = "id"; String clazz = "org.apache.nifi.controller.service.util.TestControllerService"; - ControllerServiceProvider provider = new StandardControllerServiceProvider(null, null); + ControllerServiceProvider provider = new StandardControllerServiceProvider(null, null, new StateManagerProvider() { + @Override + public StateManager getStateManager(final String componentId) { + return Mockito.mock(StateManager.class); + } + + @Override + public void shutdown() { + } + + @Override + public void enableClusterProvider() { + } + + @Override + public void disableClusterProvider() { + } + + @Override + public void onComponentRemoved(String componentId) { + } + }); ControllerServiceNode node = provider.createControllerService(clazz, id, true); proxied = node.getProxiedControllerService(); implementation = node.getControllerServiceImplementation(); @@ -68,4 +92,4 @@ public class StandardControllerServiceProviderTest { public void testCallImplementationInitialized() throws InitializationException { implementation.initialize(null); } -} +} \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java index eb0fa13f6d..ca63ba4ae7 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java @@ -28,6 +28,8 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import org.apache.nifi.components.state.StateManager; +import org.apache.nifi.components.state.StateManagerProvider; import org.apache.nifi.controller.Heartbeater; import org.apache.nifi.controller.ProcessScheduler; import org.apache.nifi.controller.ProcessorNode; @@ -49,6 +51,28 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; public class TestStandardControllerServiceProvider { + private static StateManagerProvider stateManagerProvider = new StateManagerProvider() { + @Override + public StateManager getStateManager(String componentId) { + return Mockito.mock(StateManager.class); + } + + @Override + public void shutdown() { + } + + @Override + public void enableClusterProvider() { + } + + @Override + public void disableClusterProvider() { + } + + @Override + public void onComponentRemoved(String componentId) { + } + }; @BeforeClass public static void setNiFiProps() { @@ -57,13 +81,13 @@ public class TestStandardControllerServiceProvider { private ProcessScheduler createScheduler() { final Heartbeater heartbeater = Mockito.mock(Heartbeater.class); - return new StandardProcessScheduler(heartbeater, null, null); + return new StandardProcessScheduler(heartbeater, null, null, stateManagerProvider); } @Test public void testDisableControllerService() { final ProcessScheduler scheduler = createScheduler(); - final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null); + final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null, stateManagerProvider); final ControllerServiceNode serviceNode = provider.createControllerService(ServiceB.class.getName(), "B", false); provider.enableControllerService(serviceNode); @@ -73,12 +97,12 @@ public class TestStandardControllerServiceProvider { @Test(timeout=10000) public void testEnableDisableWithReference() { final ProcessScheduler scheduler = createScheduler(); - final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null); + final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null, stateManagerProvider); final ControllerServiceNode serviceNodeB = provider.createControllerService(ServiceB.class.getName(), "B", false); final ControllerServiceNode serviceNodeA = provider.createControllerService(ServiceA.class.getName(), "A", false); - serviceNodeA.setProperty(ServiceA.OTHER_SERVICE.getName(), "B"); + serviceNodeA.setProperty(ServiceA.OTHER_SERVICE.getName(), "B", true); try { provider.enableControllerService(serviceNodeA); @@ -126,7 +150,7 @@ public class TestStandardControllerServiceProvider { } public void testEnableReferencingServicesGraph(ProcessScheduler scheduler) { - final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null); + final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null, stateManagerProvider); // build a graph of controller services with dependencies as such: // @@ -145,10 +169,10 @@ public class TestStandardControllerServiceProvider { final ControllerServiceNode serviceNode3 = provider.createControllerService(ServiceA.class.getName(), "3", false); final ControllerServiceNode serviceNode4 = provider.createControllerService(ServiceB.class.getName(), "4", false); - serviceNode1.setProperty(ServiceA.OTHER_SERVICE.getName(), "2"); - serviceNode2.setProperty(ServiceA.OTHER_SERVICE.getName(), "4"); - serviceNode3.setProperty(ServiceA.OTHER_SERVICE.getName(), "2"); - serviceNode3.setProperty(ServiceA.OTHER_SERVICE_2.getName(), "4"); + serviceNode1.setProperty(ServiceA.OTHER_SERVICE.getName(), "2", true); + serviceNode2.setProperty(ServiceA.OTHER_SERVICE.getName(), "4", true); + serviceNode3.setProperty(ServiceA.OTHER_SERVICE.getName(), "2", true); + serviceNode3.setProperty(ServiceA.OTHER_SERVICE_2.getName(), "4", true); provider.enableControllerService(serviceNode4); provider.enableReferencingServices(serviceNode4); @@ -169,7 +193,7 @@ public class TestStandardControllerServiceProvider { @Test(timeout=10000) public void testStartStopReferencingComponents() { final ProcessScheduler scheduler = createScheduler(); - final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null); + final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null, stateManagerProvider); // build a graph of reporting tasks and controller services with dependencies as such: // @@ -213,20 +237,20 @@ public class TestStandardControllerServiceProvider { final ProcessorNode procNodeA = new StandardProcessorNode(new DummyProcessor(), id1, new StandardValidationContextFactory(provider), scheduler, provider); procNodeA.getProcessor().initialize(new StandardProcessorInitializationContext(id1, null, provider)); - procNodeA.setProperty(DummyProcessor.SERVICE.getName(), "1"); + procNodeA.setProperty(DummyProcessor.SERVICE.getName(), "1", true); procNodeA.setProcessGroup(mockProcessGroup); final String id2 = UUID.randomUUID().toString(); final ProcessorNode procNodeB = new StandardProcessorNode(new DummyProcessor(), id2, new StandardValidationContextFactory(provider), scheduler, provider); procNodeB.getProcessor().initialize(new StandardProcessorInitializationContext(id2, null, provider)); - procNodeB.setProperty(DummyProcessor.SERVICE.getName(), "3"); + procNodeB.setProperty(DummyProcessor.SERVICE.getName(), "3", true); procNodeB.setProcessGroup(mockProcessGroup); - serviceNode1.setProperty(ServiceA.OTHER_SERVICE.getName(), "2"); - serviceNode2.setProperty(ServiceA.OTHER_SERVICE.getName(), "4"); - serviceNode3.setProperty(ServiceA.OTHER_SERVICE.getName(), "2"); - serviceNode3.setProperty(ServiceA.OTHER_SERVICE_2.getName(), "4"); + serviceNode1.setProperty(ServiceA.OTHER_SERVICE.getName(), "2", true); + serviceNode2.setProperty(ServiceA.OTHER_SERVICE.getName(), "4", true); + serviceNode3.setProperty(ServiceA.OTHER_SERVICE.getName(), "2", true); + serviceNode3.setProperty(ServiceA.OTHER_SERVICE_2.getName(), "4", true); provider.enableControllerService(serviceNode4); provider.enableReferencingServices(serviceNode4); @@ -280,11 +304,11 @@ public class TestStandardControllerServiceProvider { @Test public void testOrderingOfServices() { - final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(null, null); + final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(null, null, stateManagerProvider); final ControllerServiceNode serviceNode1 = provider.createControllerService(ServiceA.class.getName(), "1", false); final ControllerServiceNode serviceNode2 = provider.createControllerService(ServiceB.class.getName(), "2", false); - serviceNode1.setProperty(ServiceA.OTHER_SERVICE.getName(), "2"); + serviceNode1.setProperty(ServiceA.OTHER_SERVICE.getName(), "2", true); final Map nodeMap = new LinkedHashMap<>(); nodeMap.put("1", serviceNode1); @@ -314,7 +338,7 @@ public class TestStandardControllerServiceProvider { // add circular dependency on self. nodeMap.clear(); - serviceNode1.setProperty(ServiceA.OTHER_SERVICE_2.getName(), "1"); + serviceNode1.setProperty(ServiceA.OTHER_SERVICE_2.getName(), "1", true); nodeMap.put("1", serviceNode1); nodeMap.put("2", serviceNode2); @@ -341,8 +365,8 @@ public class TestStandardControllerServiceProvider { // like that. nodeMap.clear(); final ControllerServiceNode serviceNode3 = provider.createControllerService(ServiceA.class.getName(), "3", false); - serviceNode1.setProperty(ServiceA.OTHER_SERVICE.getName(), "3"); - serviceNode3.setProperty(ServiceA.OTHER_SERVICE.getName(), "1"); + serviceNode1.setProperty(ServiceA.OTHER_SERVICE.getName(), "3", true); + serviceNode3.setProperty(ServiceA.OTHER_SERVICE.getName(), "1", true); nodeMap.put("1", serviceNode1); nodeMap.put("3", serviceNode3); branches = StandardControllerServiceProvider.determineEnablingOrder(nodeMap); @@ -364,10 +388,10 @@ public class TestStandardControllerServiceProvider { // Add multiple completely disparate branches. nodeMap.clear(); - serviceNode1.setProperty(ServiceA.OTHER_SERVICE.getName(), "2"); + serviceNode1.setProperty(ServiceA.OTHER_SERVICE.getName(), "2", true); final ControllerServiceNode serviceNode4 = provider.createControllerService(ServiceB.class.getName(), "4", false); final ControllerServiceNode serviceNode5 = provider.createControllerService(ServiceB.class.getName(), "5", false); - serviceNode3.setProperty(ServiceA.OTHER_SERVICE.getName(), "4"); + serviceNode3.setProperty(ServiceA.OTHER_SERVICE.getName(), "4", true); nodeMap.put("1", serviceNode1); nodeMap.put("2", serviceNode2); nodeMap.put("3", serviceNode3); @@ -398,8 +422,8 @@ public class TestStandardControllerServiceProvider { // create 2 branches both dependent on the same service nodeMap.clear(); - serviceNode1.setProperty(ServiceA.OTHER_SERVICE.getName(), "2"); - serviceNode3.setProperty(ServiceA.OTHER_SERVICE.getName(), "2"); + serviceNode1.setProperty(ServiceA.OTHER_SERVICE.getName(), "2", true); + serviceNode3.setProperty(ServiceA.OTHER_SERVICE.getName(), "2", true); nodeMap.put("1", serviceNode1); nodeMap.put("2", serviceNode2); nodeMap.put("3", serviceNode3); @@ -426,7 +450,7 @@ public class TestStandardControllerServiceProvider { final ProcessorNode procNode = new StandardProcessorNode(new DummyProcessor(), UUID.randomUUID().toString(), new StandardValidationContextFactory(serviceProvider), scheduler, serviceProvider); - final ProcessGroup group = new StandardProcessGroup(UUID.randomUUID().toString(), serviceProvider, scheduler, null, null); + final ProcessGroup group = new StandardProcessGroup(UUID.randomUUID().toString(), serviceProvider, scheduler, null, null, null); group.addProcessor(procNode); procNode.setProcessGroup(group); @@ -436,7 +460,7 @@ public class TestStandardControllerServiceProvider { @Test public void testEnableReferencingComponents() { final ProcessScheduler scheduler = createScheduler(); - final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(null, null); + final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(null, null, stateManagerProvider); final ControllerServiceNode serviceNode = provider.createControllerService(ServiceA.class.getName(), "1", false); final ProcessorNode procNode = createProcessor(scheduler, provider); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/state/providers/AbstractTestStateProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/state/providers/AbstractTestStateProvider.java new file mode 100644 index 0000000000..d724be0815 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/state/providers/AbstractTestStateProvider.java @@ -0,0 +1,153 @@ +/* + * 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 static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import org.apache.nifi.components.state.StateMap; +import org.apache.nifi.components.state.StateProvider; +import org.junit.Test; + + +/** + *

+ * Abstract class that provides a suite of test for State Providers. Each State Provider implementation can simply extend this class, + * implement the getProvider() method, and have the entire suite of tests run against the provider. + *

+ * + *

+ * It is recommended that implementations create a new provider in a method annotated with @Before and cleanup in a method annotated with @After. + *

+ */ +public abstract class AbstractTestStateProvider { + protected final String componentId = "111111111-1111-1111-1111-111111111111"; + + + @Test + public void testSetAndGet() throws IOException { + getProvider().setState(Collections.singletonMap("testSetAndGet", "value"), componentId); + assertEquals("value", getProvider().getState(componentId).get("testSetAndGet")); + } + + @Test + public void testReplaceSuccessful() throws IOException { + final String key = "testReplaceSuccessful"; + final StateProvider provider = getProvider(); + + StateMap map = provider.getState(componentId); + assertNotNull(map); + assertEquals(-1, map.getVersion()); + + assertNotNull(map.toMap()); + assertTrue(map.toMap().isEmpty()); + provider.setState(Collections.singletonMap(key, "value1"), componentId); + + map = provider.getState(componentId); + assertNotNull(map); + assertEquals(0, map.getVersion()); + assertEquals("value1", map.get(key)); + assertEquals("value1", map.toMap().get(key)); + + final Map newMap = new HashMap<>(map.toMap()); + newMap.put(key, "value2"); + assertTrue(provider.replace(map, newMap, componentId)); + + map = provider.getState(componentId); + assertEquals("value2", map.get(key)); + assertEquals(1L, map.getVersion()); + } + + @Test + public void testReplaceWithWrongVersion() throws IOException { + final String key = "testReplaceWithWrongVersion"; + final StateProvider provider = getProvider(); + provider.setState(Collections.singletonMap(key, "value1"), componentId); + + StateMap stateMap = provider.getState(componentId); + assertNotNull(stateMap); + assertEquals("value1", stateMap.get(key)); + assertEquals(0, stateMap.getVersion()); + + provider.setState(Collections.singletonMap(key, "intermediate value"), componentId); + + assertFalse(provider.replace(stateMap, Collections.singletonMap(key, "value2"), componentId)); + stateMap = provider.getState(componentId); + assertEquals(key, stateMap.toMap().keySet().iterator().next()); + assertEquals(1, stateMap.toMap().size()); + assertEquals("intermediate value", stateMap.get(key)); + assertEquals(1, stateMap.getVersion()); + } + + + @Test + public void testToMap() throws IOException { + final String key = "testKeySet"; + final StateProvider provider = getProvider(); + Map map = provider.getState(componentId).toMap(); + assertNotNull(map); + assertTrue(map.isEmpty()); + + provider.setState(Collections.singletonMap(key, "value"), componentId); + map = provider.getState(componentId).toMap(); + assertNotNull(map); + assertEquals(1, map.size()); + assertEquals("value", map.get(key)); + + provider.setState(Collections. emptyMap(), componentId); + + final StateMap stateMap = provider.getState(componentId); + map = stateMap.toMap(); + assertNotNull(map); + assertTrue(map.isEmpty()); + assertEquals(1, stateMap.getVersion()); + } + + @Test + public void testClear() throws IOException { + final StateProvider provider = getProvider(); + StateMap stateMap = provider.getState(componentId); + assertNotNull(stateMap); + assertEquals(-1L, stateMap.getVersion()); + assertTrue(stateMap.toMap().isEmpty()); + + provider.setState(Collections.singletonMap("testClear", "value"), componentId); + + stateMap = provider.getState(componentId); + assertNotNull(stateMap); + assertEquals(0, stateMap.getVersion()); + assertEquals("value", stateMap.get("testClear")); + + provider.clear(componentId); + + stateMap = provider.getState(componentId); + assertNotNull(stateMap); + assertEquals(1L, stateMap.getVersion()); + assertTrue(stateMap.toMap().isEmpty()); + } + + + protected abstract StateProvider getProvider(); +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/state/providers/local/TestWriteAheadLocalStateProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/state/providers/local/TestWriteAheadLocalStateProvider.java new file mode 100644 index 0000000000..9f7c4c9825 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/state/providers/local/TestWriteAheadLocalStateProvider.java @@ -0,0 +1,90 @@ +/* + * 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.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; + +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.state.StateProvider; +import org.apache.nifi.components.state.StateProviderInitializationContext; +import org.apache.nifi.controller.state.StateMapUpdate; +import org.apache.nifi.controller.state.providers.AbstractTestStateProvider; +import org.junit.After; +import org.junit.Before; +import org.wali.WriteAheadRepository; + +public class TestWriteAheadLocalStateProvider extends AbstractTestStateProvider { + private StateProvider provider; + private WriteAheadRepository wal; + + @Before + public void setup() throws IOException { + provider = new WriteAheadLocalStateProvider(); + + final Map properties = new HashMap<>(); + properties.put(WriteAheadLocalStateProvider.PATH, new StandardPropertyValue("target/local-state-provider/" + UUID.randomUUID().toString(), null)); + + provider.initialize(new StateProviderInitializationContext() { + @Override + public String getIdentifier() { + return "Unit Test Provider Initialization Context"; + } + + @Override + public Map getProperties() { + return Collections.unmodifiableMap(properties); + } + + @Override + public PropertyValue getProperty(final PropertyDescriptor property) { + final PropertyValue prop = properties.get(property); + if (prop == null) { + return new StandardPropertyValue(null, null); + } + return prop; + } + + @Override + public SSLContext getSSLContext() { + return null; + } + }); + } + + @After + public void cleanup() throws IOException { + provider.onComponentRemoved(componentId); + + if (wal != null) { + wal.shutdown(); + } + } + + @Override + protected StateProvider getProvider() { + return provider; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/state/providers/zookeeper/TestZooKeeperStateProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/state/providers/zookeeper/TestZooKeeperStateProvider.java new file mode 100644 index 0000000000..b84128572d --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/state/providers/zookeeper/TestZooKeeperStateProvider.java @@ -0,0 +1,106 @@ +/* + * 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.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import javax.net.ssl.SSLContext; + +import org.apache.curator.test.TestingServer; +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.StateProvider; +import org.apache.nifi.components.state.StateProviderInitializationContext; +import org.apache.nifi.controller.state.providers.AbstractTestStateProvider; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; + +public class TestZooKeeperStateProvider extends AbstractTestStateProvider { + + private StateProvider provider; + private static TestingServer zkServer; + + @BeforeClass + public static void createZooKeeper() throws Exception { + zkServer = new TestingServer(2181, true); + zkServer.start(); + } + + @AfterClass + public static void tearDown() throws IOException { + if (zkServer != null) { + zkServer.stop(); + zkServer.close(); + } + } + + @Before + public void setup() throws Exception { + final Map properties = new HashMap<>(); + properties.put(ZooKeeperStateProvider.CONNECTION_STRING, new StandardPropertyValue(zkServer.getConnectString(), null)); + properties.put(ZooKeeperStateProvider.SESSION_TIMEOUT, new StandardPropertyValue("3 secs", null)); + properties.put(ZooKeeperStateProvider.ROOT_NODE, new StandardPropertyValue("/nifi/team1/testing", null)); + + provider = new ZooKeeperStateProvider(); + provider.initialize(new StateProviderInitializationContext() { + @Override + public String getIdentifier() { + return "Unit Test Provider Initialization Context"; + } + + @Override + public Map getProperties() { + return Collections.unmodifiableMap(properties); + } + + @Override + public PropertyValue getProperty(final PropertyDescriptor property) { + final PropertyValue prop = properties.get(property); + if (prop == null) { + return new StandardPropertyValue(null, null); + } + return prop; + } + + @Override + public SSLContext getSSLContext() { + return null; + } + }); + + provider.enable(); + } + + @After + public void clear() throws IOException { + getProvider().onComponentRemoved(componentId); + getProvider().disable(); + getProvider().shutdown(); + } + + @Override + protected StateProvider getProvider() { + return provider; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties index 00e1274309..4a665d97a2 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties @@ -33,6 +33,20 @@ nifi.nar.library.directory=${nifi.nar.library.directory} nifi.nar.working.directory=${nifi.nar.working.directory} nifi.documentation.working.directory=${nifi.documentation.working.directory} +#################### +# State Management # +#################### +nifi.state.management.configuration.file=${nifi.state.management.configuration.file} +# The ID of the local state provider +nifi.state.management.provider.local=${nifi.state.management.provider.local} +# The ID of the cluster-wide state provider. This will be ignored if NiFi is not clustered but must be populated if running in a cluster. +nifi.state.management.provider.cluster=${nifi.state.management.provider.cluster} +# Specifies whether or not this instance of NiFi should run an embedded ZooKeeper server +nifi.state.management.embedded.zookeeper.start=${nifi.state.management.embedded.zookeeper.start} +# Properties file that provides the ZooKeeper properties to use if is set to true +nifi.state.management.embedded.zookeeper.properties=${nifi.state.management.embedded.zookeeper.properties} + + # H2 Settings nifi.database.directory=${nifi.database.directory} nifi.h2.url.append=${nifi.h2.url.append} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/state-management.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/state-management.xml new file mode 100644 index 0000000000..ba5259c80e --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/state-management.xml @@ -0,0 +1,52 @@ + + + + + + + local-provider + org.apache.nifi.controller.state.providers.local.WriteAheadLocalStateProvider + ./state + + + + + zk-provider + org.apache.nifi.controller.state.providers.zookeeper.ZooKeeperStateProvider + + /nifi + 30 seconds + + \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/zookeeper.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/zookeeper.properties new file mode 100644 index 0000000000..55afc94c70 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/zookeeper.properties @@ -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. +# +# +# + +clientPort=2181 +initLimit=10 +autopurge.purgeInterval=24 +syncLimit=5 +tickTime=2000 +dataDir=./state/zookeeper +autopurge.snapRetainCount=30 +server.1= \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.java index 7d0ffab4b2..ef7a61cd0f 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.java @@ -133,7 +133,7 @@ public class ClusterManagerServerProtocol implements ServerProtocol { continue; } - dos.writeUTF(nodeInfo.getHostname()); + dos.writeUTF(nodeInfo.getSiteToSiteHostname()); dos.writeInt(nodeInfo.getSiteToSitePort()); dos.writeBoolean(nodeInfo.isSiteToSiteSecure()); dos.writeInt(nodeInfo.getTotalFlowFiles()); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java index 486e6d77ff..0676668ff5 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java @@ -241,9 +241,9 @@ public class StandardControllerServiceDAO extends ComponentDAO implements Contro final String propName = entry.getKey(); final String propVal = entry.getValue(); if (isNotNull(propName) && propVal == null) { - controllerService.removeProperty(propName); + controllerService.removeProperty(propName, true); } else if (isNotNull(propName)) { - controllerService.setProperty(propName, propVal); + controllerService.setProperty(propName, propVal, true); } } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java index e52a476c95..81d882c8c8 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java @@ -171,9 +171,9 @@ public class StandardProcessorDAO extends ComponentDAO implements ProcessorDAO { final String propName = entry.getKey(); final String propVal = entry.getValue(); if (isNotNull(propName) && propVal == null) { - processor.removeProperty(propName); + processor.removeProperty(propName, true); } else if (isNotNull(propName)) { - processor.setProperty(propName, propVal); + processor.setProperty(propName, propVal, true); } } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java index ffe606a80a..38d793dfe7 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java @@ -285,9 +285,9 @@ public class StandardReportingTaskDAO extends ComponentDAO implements ReportingT final String propName = entry.getKey(); final String propVal = entry.getValue(); if (isNotNull(propName) && propVal == null) { - reportingTask.removeProperty(propName); + reportingTask.removeProperty(propName, true); } else if (isNotNull(propName)) { - reportingTask.setProperty(propName, propVal); + reportingTask.setProperty(propName, propVal, true); } } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/access-control/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/access-control/nifi.properties index 10db651eb8..8c54f249ab 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/access-control/nifi.properties +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/access-control/nifi.properties @@ -29,6 +29,13 @@ nifi.ui.autorefresh.interval=30 sec nifi.nar.library.directory=target/test-classes/access-control/lib nifi.nar.working.directory=target/test-classes/access-control/nar +nifi.state.management.configuration.file=target/test-classes/access-control/state-management.xml +nifi.state.management.embedded.zookeeper.start=false +nifi.state.management.embedded.zookeeper.properties= +nifi.state.management.embedded.zookeeper.max.instances=3 +nifi.state.management.provider.local=local-provider +nifi.state.management.provider.cluster= + # H2 Settings nifi.database.directory=target/test-classes/database_repository nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/access-control/state-management.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/access-control/state-management.xml new file mode 100644 index 0000000000..1714c7dcbb --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/access-control/state-management.xml @@ -0,0 +1,38 @@ + + + + + + + + + local-provider + org.apache.nifi.controller.state.providers.local.WriteAheadLocalStateProvider + target/test-classes/access-control/state-management + + \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/ExtractHL7Attributes.java b/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/ExtractHL7Attributes.java index 3a6ac79d8a..e49a8bea4f 100644 --- a/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/ExtractHL7Attributes.java +++ b/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/ExtractHL7Attributes.java @@ -55,6 +55,7 @@ import ca.uhn.hl7v2.model.Structure; import ca.uhn.hl7v2.model.Type; import ca.uhn.hl7v2.model.Varies; import ca.uhn.hl7v2.parser.PipeParser; +import ca.uhn.hl7v2.validation.ValidationContext; import ca.uhn.hl7v2.validation.impl.ValidationContextFactory; @SideEffectFree @@ -120,7 +121,7 @@ public class ExtractHL7Attributes extends AbstractProcessor { @SuppressWarnings("resource") final HapiContext hapiContext = new DefaultHapiContext(); - hapiContext.setValidationContext(ValidationContextFactory.noValidation()); + hapiContext.setValidationContext((ValidationContext) ValidationContextFactory.noValidation()); final PipeParser parser = hapiContext.getPipeParser(); final String hl7Text = new String(buffer, charset); diff --git a/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/RouteHL7.java b/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/RouteHL7.java index 26e8bb6f5d..ba90370cdb 100644 --- a/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/RouteHL7.java +++ b/nifi-nar-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/RouteHL7.java @@ -161,7 +161,7 @@ public class RouteHL7 extends AbstractProcessor { @SuppressWarnings("resource") final HapiContext hapiContext = new DefaultHapiContext(); - hapiContext.setValidationContext(ValidationContextFactory.noValidation()); + hapiContext.setValidationContext((ca.uhn.hl7v2.validation.ValidationContext) ValidationContextFactory.noValidation()); final PipeParser parser = hapiContext.getPipeParser(); final String hl7Text = new String(buffer, charset); diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafka.java index 08f5aa70e4..802f8891fb 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafka.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/test/EmbeddedKafka.java @@ -166,8 +166,7 @@ public class EmbeddedKafka { ServerConfig configuration = new ServerConfig(); configuration.readFrom(quorumConfiguration); - FileTxnSnapLog txnLog = new FileTxnSnapLog(new File(configuration.getDataLogDir()), - new File(configuration.getDataDir())); + FileTxnSnapLog txnLog = new FileTxnSnapLog(new File(configuration.getDataLogDir()), new File(configuration.getDataDir())); zkServer.setTxnLogFactory(txnLog); zkServer.setTickTime(configuration.getTickTime()); diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractListProcessor.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractListProcessor.java index e5924832bb..20969c9c7a 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractListProcessor.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractListProcessor.java @@ -19,14 +19,12 @@ package org.apache.nifi.processors.standard; import java.io.File; import java.io.FileInputStream; -import java.io.FileOutputStream; import java.io.IOException; import java.io.OutputStream; import java.nio.charset.StandardCharsets; import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.Date; +import java.util.Collection; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -34,9 +32,13 @@ import java.util.Properties; import java.util.Set; import org.apache.nifi.annotation.behavior.TriggerSerially; +import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.annotation.notification.OnPrimaryNodeStateChange; import org.apache.nifi.annotation.notification.PrimaryNodeState; import org.apache.nifi.components.PropertyDescriptor; +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.distributed.cache.client.Deserializer; import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient; import org.apache.nifi.distributed.cache.client.Serializer; @@ -50,7 +52,6 @@ import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processors.standard.util.EntityListing; import org.apache.nifi.processors.standard.util.ListableEntity; -import org.codehaus.jackson.JsonGenerationException; import org.codehaus.jackson.JsonNode; import org.codehaus.jackson.JsonParseException; import org.codehaus.jackson.map.JsonMappingException; @@ -157,9 +158,13 @@ public abstract class AbstractListProcessor extends Ab .build(); - private volatile Long lastListingTime = null; private volatile Set latestIdentifiersListed = new HashSet<>(); + private volatile Long lastListingTime = null; private volatile boolean electedPrimaryNode = false; + private volatile boolean resetListing = false; + + static final String TIMESTAMP = "timestamp"; + static final String IDENTIFIER_PREFIX = "id"; protected File getPersistenceFile() { return new File("conf/state/" + getIdentifier()); @@ -177,6 +182,7 @@ public abstract class AbstractListProcessor extends Ab if (isListingResetNecessary(descriptor)) { lastListingTime = null; // clear lastListingTime so that we have to fetch new time latestIdentifiersListed = new HashSet<>(); + resetListing = true; } } @@ -187,10 +193,6 @@ public abstract class AbstractListProcessor extends Ab return relationships; } - protected String getKey(final String directory) { - return getIdentifier() + ".lastListingTime." + directory; - } - @OnPrimaryNodeStateChange public void onPrimaryNodeChange(final PrimaryNodeState newState) { if (newState == PrimaryNodeState.ELECTED_PRIMARY_NODE) { @@ -198,6 +200,111 @@ public abstract class AbstractListProcessor extends Ab } } + @OnScheduled + public final void updateState(final ProcessContext context) throws IOException { + final String path = getPath(context); + final DistributedMapCacheClient client = context.getProperty(DISTRIBUTED_CACHE_SERVICE).asControllerService(DistributedMapCacheClient.class); + + // Check if state already exists for this path. If so, we have already migrated the state. + final StateMap stateMap = context.getStateManager().getState(Scope.CLUSTER); + if (stateMap.getVersion() == -1L) { + try { + // Migrate state from the old way of managing state (distributed cache service and local file) + // to the new mechanism (State Manager). + migrateState(path, client, context.getStateManager()); + } catch (final IOException ioe) { + throw new IOException("Failed to properly migrate state to State Manager", ioe); + } + } + + // delete the local file, since it is no longer needed + final File localFile = new File(path); + if (localFile.exists() && !!localFile.delete()) { + getLogger().warn("Migrated state but failed to delete local persistence file"); + } + + // remove entry from Distributed cache server + if (client != null) { + try { + client.remove(path, new StringSerDe()); + } catch (final IOException ioe) { + getLogger().warn("Failed to remove entry from Distributed Cache Service. However, the state has already been migrated to use the new " + + "State Management service, so the Distributed Cache Service is no longer needed."); + } + } + + if (resetListing) { + context.getStateManager().clear(Scope.CLUSTER); + resetListing = false; + } + } + + /** + * This processor used to use the DistributedMapCacheClient in order to store cluster-wide state, before the introduction of + * the StateManager. This method will migrate state from that DistributedMapCacheClient, or from a local file, to the StateManager, + * if any state already exists + * + * @param path the path to migrate state for + * @param client the DistributedMapCacheClient that is capable of obtaining the current state + * @param stateManager the StateManager to use in order to store the new state + * @throws IOException if unable to retrieve or store the state + */ + private void migrateState(final String path, final DistributedMapCacheClient client, final StateManager stateManager) throws IOException { + Long minTimestamp = null; + final Set latestIdentifiersListed = new HashSet<>(); + + // Retrieve state from Distributed Cache Client + if (client != null) { + final StringSerDe serde = new StringSerDe(); + final String serializedState = client.get(getKey(path), serde, serde); + if (serializedState != null && !serializedState.isEmpty()) { + final EntityListing listing = deserialize(serializedState); + minTimestamp = listing.getLatestTimestamp().getTime(); + latestIdentifiersListed.addAll(listing.getMatchingIdentifiers()); + } + } + + // Retrieve state from locally persisted file + final File persistenceFile = getPersistenceFile(); + if (persistenceFile.exists()) { + final Properties props = new Properties(); + + try (final FileInputStream fis = new FileInputStream(persistenceFile)) { + props.load(fis); + } + + final String locallyPersistedValue = props.getProperty(path); + if (locallyPersistedValue != null) { + final EntityListing listing = deserialize(locallyPersistedValue); + final long localTimestamp = listing.getLatestTimestamp().getTime(); + if (minTimestamp == null || localTimestamp > minTimestamp) { + minTimestamp = localTimestamp; + latestIdentifiersListed.clear(); + latestIdentifiersListed.addAll(listing.getMatchingIdentifiers()); + } + } + } + + if (minTimestamp != null) { + persist(minTimestamp, latestIdentifiersListed, stateManager); + } + } + + private void persist(final long timestamp, final Collection identifiers, final StateManager stateManager) throws IOException { + final Map updatedState = new HashMap<>(identifiers.size() + 1); + updatedState.put(TIMESTAMP, String.valueOf(timestamp)); + int counter = 0; + for (final String identifier : identifiers) { + final String index = String.valueOf(++counter); + updatedState.put(IDENTIFIER_PREFIX + "." + index, identifier); + } + stateManager.setState(updatedState, Scope.CLUSTER); + } + + protected String getKey(final String directory) { + return getIdentifier() + ".lastListingTime." + directory; + } + private EntityListing deserialize(final String serializedState) throws JsonParseException, JsonMappingException, IOException { final ObjectMapper mapper = new ObjectMapper(); final JsonNode jsonNode = mapper.readTree(serializedState); @@ -205,152 +312,34 @@ public abstract class AbstractListProcessor extends Ab } - private Long getMinTimestamp(final String directory, final DistributedMapCacheClient client) throws IOException { - // Determine the timestamp for the last file that we've listed. - Long minTimestamp = lastListingTime; - if (minTimestamp == null || electedPrimaryNode) { - // We haven't yet restored any state from local or distributed state - or it's been at least a minute since - // we have performed a listing. In this case, - // First, attempt to get timestamp from distributed cache service. - if (client != null) { - try { - final StringSerDe serde = new StringSerDe(); - final String serializedState = client.get(getKey(directory), serde, serde); - if (serializedState == null || serializedState.isEmpty()) { - minTimestamp = null; - this.latestIdentifiersListed = Collections.emptySet(); - } else { - final EntityListing listing = deserialize(serializedState); - this.lastListingTime = listing.getLatestTimestamp().getTime(); - minTimestamp = listing.getLatestTimestamp().getTime(); - this.latestIdentifiersListed = new HashSet<>(listing.getMatchingIdentifiers()); - } - - this.lastListingTime = minTimestamp; - electedPrimaryNode = false; // no requirement to pull an update from the distributed cache anymore. - } catch (final IOException ioe) { - throw ioe; - } - } - - // Check the persistence file. We want to use the latest timestamp that we have so that - // we don't duplicate data. - try { - final File persistenceFile = getPersistenceFile(); - if (persistenceFile.exists()) { - try (final FileInputStream fis = new FileInputStream(persistenceFile)) { - final Properties props = new Properties(); - props.load(fis); - - // get the local timestamp for this directory, if it exists. - final String locallyPersistedValue = props.getProperty(directory); - if (locallyPersistedValue != null) { - final EntityListing listing = deserialize(locallyPersistedValue); - final long localTimestamp = listing.getLatestTimestamp().getTime(); - - // If distributed state doesn't have an entry or the local entry is later than the distributed state, - // update the distributed state so that we are in sync. - if (client != null && (minTimestamp == null || localTimestamp > minTimestamp)) { - minTimestamp = localTimestamp; - - // Our local persistence file shows a later time than the Distributed service. - // Update the distributed service to match our local state. - try { - final StringSerDe serde = new StringSerDe(); - client.put(getKey(directory), locallyPersistedValue, serde, serde); - } catch (final IOException ioe) { - getLogger().warn("Local timestamp for {} is {}, which is later than Distributed state but failed to update Distributed " - + "state due to {}. If a new node performs Listing, data duplication may occur", - new Object[] {directory, locallyPersistedValue, ioe}); - } - } - } - } - } - } catch (final IOException ioe) { - getLogger().warn("Failed to recover local state due to {}. Assuming that the state from the distributed cache is correct.", ioe); - } - } - - return minTimestamp; - } - - - private String serializeState(final List entities) throws JsonGenerationException, JsonMappingException, IOException { - // we need to keep track of all files that we pulled in that had a modification time equal to - // lastListingTime so that we can avoid pulling those files in again. We can't just ignore any files - // that have a mod time equal to that timestamp because more files may come in with the same timestamp - // later in the same millisecond. - if (entities.isEmpty()) { - return null; - } else { - final List sortedEntities = new ArrayList<>(entities); - Collections.sort(sortedEntities, new Comparator() { - @Override - public int compare(final ListableEntity o1, final ListableEntity o2) { - return Long.compare(o1.getTimestamp(), o2.getTimestamp()); - } - }); - - final long latestListingModTime = sortedEntities.get(sortedEntities.size() - 1).getTimestamp(); - final Set idsWithTimestampEqualToListingTime = new HashSet<>(); - for (int i = sortedEntities.size() - 1; i >= 0; i--) { - final ListableEntity entity = sortedEntities.get(i); - if (entity.getTimestamp() == latestListingModTime) { - idsWithTimestampEqualToListingTime.add(entity.getIdentifier()); - } - } - - this.latestIdentifiersListed = idsWithTimestampEqualToListingTime; - - final EntityListing listing = new EntityListing(); - listing.setLatestTimestamp(new Date(latestListingModTime)); - final Set ids = new HashSet<>(); - for (final String id : idsWithTimestampEqualToListingTime) { - ids.add(id); - } - listing.setMatchingIdentifiers(ids); - - final ObjectMapper mapper = new ObjectMapper(); - final String serializedState = mapper.writerWithType(EntityListing.class).writeValueAsString(listing); - return serializedState; - } - } - - protected void persistLocalState(final String path, final String serializedState) throws IOException { - // we need to keep track of all files that we pulled in that had a modification time equal to - // lastListingTime so that we can avoid pulling those files in again. We can't just ignore any files - // that have a mod time equal to that timestamp because more files may come in with the same timestamp - // later in the same millisecond. - final File persistenceFile = getPersistenceFile(); - final File dir = persistenceFile.getParentFile(); - if (!dir.exists() && !dir.mkdirs()) { - throw new IOException("Could not create directory " + dir.getAbsolutePath() + " in order to save local state"); - } - - final Properties props = new Properties(); - if (persistenceFile.exists()) { - try (final FileInputStream fis = new FileInputStream(persistenceFile)) { - props.load(fis); - } - } - - props.setProperty(path, serializedState); - - try (final FileOutputStream fos = new FileOutputStream(persistenceFile)) { - props.store(fos, null); - } - } - - @Override public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { - final String path = getPath(context); - final DistributedMapCacheClient client = context.getProperty(DISTRIBUTED_CACHE_SERVICE).asControllerService(DistributedMapCacheClient.class); - - final Long minTimestamp; + Long minTimestamp = lastListingTime; try { - minTimestamp = getMinTimestamp(path, client); + // We need to fetch the state from the cluster if we don't yet know the last listing time, + // or if we were just elected the primary node + if (this.lastListingTime == null || electedPrimaryNode) { + final StateMap stateMap = context.getStateManager().getState(Scope.CLUSTER); + final Map stateValues = stateMap.toMap(); + final String timestamp = stateValues.get(TIMESTAMP); + + if (timestamp == null) { + minTimestamp = 0L; + latestIdentifiersListed.clear(); + } else { + minTimestamp = this.lastListingTime = Long.parseLong(timestamp); + latestIdentifiersListed.clear(); + for (final Map.Entry entry : stateValues.entrySet()) { + final String key = entry.getKey(); + final String value = entry.getValue(); + if (TIMESTAMP.equals(key)) { + continue; + } + + latestIdentifiersListed.add(value); + } + } + } } catch (final IOException ioe) { getLogger().error("Failed to retrieve timestamp of last listing from Distributed Cache Service. Will not perform listing until this is accomplished."); context.yield(); @@ -403,32 +392,19 @@ public abstract class AbstractListProcessor extends Ab // previously Primary Node left off. // We also store the state locally so that if the node is restarted, and the node cannot contact // the distributed state cache, the node can continue to run (if it is primary node). - String serializedState = null; + final Set identifiers = new HashSet<>(entityList.size()); try { - serializedState = serializeState(entityList); - } catch (final Exception e) { - getLogger().error("Failed to serialize state due to {}", new Object[] {e}); - } - - if (serializedState != null) { - // Save our state locally. - try { - persistLocalState(path, serializedState); - } catch (final IOException ioe) { - getLogger().warn("Unable to save state locally. If the node is restarted now, data may be duplicated. Failure is due to {}", ioe); - } - - // Attempt to save state to remote server. - if (client != null) { - try { - client.put(getKey(path), serializedState, new StringSerDe(), new StringSerDe()); - } catch (final IOException ioe) { - getLogger().warn("Unable to communicate with distributed cache server due to {}. Persisting state locally instead.", ioe); - } + for (final T entity : entityList) { + identifiers.add(entity.getIdentifier()); } + persist(latestListingTimestamp, identifiers, context.getStateManager()); + } catch (final IOException ioe) { + getLogger().warn("Unable to save state due to {}. If NiFi restarted before state is saved, or " + + "if another node begins executing this Processor, data duplication may occur.", ioe); } lastListingTime = latestListingTimestamp; + latestIdentifiersListed = identifiers; } else { getLogger().debug("There is no data to list. Yielding."); context.yield(); diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java index 5eab704e81..d3eb5154e8 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java @@ -18,7 +18,6 @@ package org.apache.nifi.processors.standard; import java.io.File; import java.io.FileInputStream; -import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; import java.net.URI; @@ -29,22 +28,16 @@ import java.security.KeyStoreException; import java.security.NoSuchAlgorithmException; import java.security.UnrecoverableKeyException; import java.security.cert.CertificateException; -import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.Date; +import java.util.HashMap; import java.util.HashSet; import java.util.List; -import java.util.Locale; -import java.util.Properties; +import java.util.Map; import java.util.Set; -import java.util.TimeZone; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; -import java.util.concurrent.locks.ReentrantReadWriteLock; -import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock; -import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Pattern; import javax.net.ssl.SSLContext; @@ -76,11 +69,12 @@ import org.apache.nifi.annotation.behavior.WritesAttribute; import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.lifecycle.OnRemoved; -import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.components.PropertyDescriptor; 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.StateMap; import org.apache.nifi.expression.AttributeExpression; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; @@ -202,30 +196,14 @@ public class GetHTTP extends AbstractSessionFactoryProcessor { public static final String LAST_MODIFIED_DATE_PATTERN_RFC1123 = "EEE, dd MMM yyyy HH:mm:ss zzz"; // package access to enable unit testing - static final String UNINITIALIZED_LAST_MODIFIED_VALUE; - - private static final String HTTP_CACHE_FILE_PREFIX = "conf/.httpCache-"; - static final String ETAG = "ETag"; - static final String LAST_MODIFIED = "LastModified"; - static { - final SimpleDateFormat sdf = new SimpleDateFormat(LAST_MODIFIED_DATE_PATTERN_RFC1123, Locale.US); - sdf.setTimeZone(TimeZone.getTimeZone("GMT")); - UNINITIALIZED_LAST_MODIFIED_VALUE = sdf.format(new Date(1L)); - } - final AtomicReference lastModifiedRef = new AtomicReference<>(UNINITIALIZED_LAST_MODIFIED_VALUE); - final AtomicReference entityTagRef = new AtomicReference<>(""); - // end private Set relationships; private List properties; - private volatile long timeToPersist = 0; - private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); - private final ReadLock readLock = lock.readLock(); - private final WriteLock writeLock = lock.writeLock(); + private final AtomicBoolean clearState = new AtomicBoolean(false); @Override protected void init(final ProcessorInitializationContext context) { @@ -247,16 +225,6 @@ public class GetHTTP extends AbstractSessionFactoryProcessor { properties.add(PROXY_HOST); properties.add(PROXY_PORT); this.properties = Collections.unmodifiableList(properties); - - // load etag and lastModified from file - final File httpCache = new File(HTTP_CACHE_FILE_PREFIX + getIdentifier()); - try (FileInputStream fis = new FileInputStream(httpCache)) { - final Properties props = new Properties(); - props.load(fis); - entityTagRef.set(props.getProperty(ETAG)); - lastModifiedRef.set(props.getProperty(LAST_MODIFIED)); - } catch (final IOException swallow) { - } } @Override @@ -271,28 +239,13 @@ public class GetHTTP extends AbstractSessionFactoryProcessor { @Override public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) { - entityTagRef.set(""); - lastModifiedRef.set(UNINITIALIZED_LAST_MODIFIED_VALUE); + clearState.set(true); } - @OnStopped - public void onStopped() { - final File httpCache = new File(HTTP_CACHE_FILE_PREFIX + getIdentifier()); - try (FileOutputStream fos = new FileOutputStream(httpCache)) { - final Properties props = new Properties(); - props.setProperty(ETAG, entityTagRef.get()); - props.setProperty(LAST_MODIFIED, lastModifiedRef.get()); - props.store(fos, "GetHTTP file modification values"); - } catch (final IOException swallow) { - } - - } - - @OnRemoved - public void onRemoved() { - final File httpCache = new File(HTTP_CACHE_FILE_PREFIX + getIdentifier()); - if (httpCache.exists()) { - httpCache.delete(); + @OnScheduled + public void onScheduled(final ProcessContext context) throws IOException { + if (clearState.getAndSet(false)) { + context.getStateManager().clear(Scope.LOCAL); } } @@ -444,8 +397,20 @@ public class GetHTTP extends AbstractSessionFactoryProcessor { final HttpGet get = new HttpGet(url); get.setConfig(requestConfigBuilder.build()); - get.addHeader(HEADER_IF_MODIFIED_SINCE, lastModifiedRef.get()); - get.addHeader(HEADER_IF_NONE_MATCH, entityTagRef.get()); + try { + final StateMap stateMap = context.getStateManager().getState(Scope.LOCAL); + final String lastModified = stateMap.get(LAST_MODIFIED); + if (lastModified != null) { + get.addHeader(HEADER_IF_MODIFIED_SINCE, lastModified); + } + + final String etag = stateMap.get(ETAG); + if (etag != null) { + get.addHeader(HEADER_IF_NONE_MATCH, etag); + } + } catch (final IOException ioe) { + throw new ProcessException(ioe); + } final String accept = context.getProperty(ACCEPT_CONTENT_TYPE).getValue(); if (accept != null) { @@ -492,41 +457,20 @@ public class GetHTTP extends AbstractSessionFactoryProcessor { session.transfer(flowFile, REL_SUCCESS); logger.info("Successfully received {} from {} at a rate of {}; transferred to success", new Object[]{flowFile, url, dataRate}); session.commit(); + + final Map updatedState = new HashMap<>(2); final Header lastModified = response.getFirstHeader(HEADER_LAST_MODIFIED); if (lastModified != null) { - lastModifiedRef.set(lastModified.getValue()); + updatedState.put(LAST_MODIFIED, lastModified.getValue()); } final Header etag = response.getFirstHeader(HEADER_ETAG); if (etag != null) { - entityTagRef.set(etag.getValue()); + updatedState.put(ETAG, etag.getValue()); } - if ((etag != null || lastModified != null) && readLock.tryLock()) { - try { - if (timeToPersist < System.currentTimeMillis()) { - readLock.unlock(); - writeLock.lock(); - try { - if (timeToPersist < System.currentTimeMillis()) { - timeToPersist = System.currentTimeMillis() + PERSISTENCE_INTERVAL_MSEC; - final File httpCache = new File(HTTP_CACHE_FILE_PREFIX + getIdentifier()); - try (FileOutputStream fos = new FileOutputStream(httpCache)) { - final Properties props = new Properties(); - props.setProperty(ETAG, entityTagRef.get()); - props.setProperty(LAST_MODIFIED, lastModifiedRef.get()); - props.store(fos, "GetHTTP file modification values"); - } catch (final IOException e) { - getLogger().error("Failed to persist ETag and LastMod due to " + e, e); - } - } - } finally { - readLock.lock(); - writeLock.unlock(); - } - } - } finally { - readLock.unlock(); - } + + if (!updatedState.isEmpty()) { + context.getStateManager().setState(updatedState, Scope.LOCAL); } } catch (final IOException e) { context.yield(); diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAbstractListProcessor.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAbstractListProcessor.java index ba84939d0e..37316b699d 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAbstractListProcessor.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAbstractListProcessor.java @@ -26,8 +26,10 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.UUID; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.state.Scope; import org.apache.nifi.controller.AbstractControllerService; import org.apache.nifi.distributed.cache.client.Deserializer; import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient; @@ -35,6 +37,7 @@ import org.apache.nifi.distributed.cache.client.Serializer; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processors.standard.util.ListableEntity; import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.state.MockStateManager; import org.apache.nifi.util.TestRunner; import org.apache.nifi.util.TestRunners; import org.junit.Test; @@ -96,7 +99,7 @@ public class TestAbstractListProcessor { } @Test - public void testStateStoredInDistributedService() throws InitializationException { + public void testStateStoredInClusterStateManagement() throws InitializationException { final ConcreteListProcessor proc = new ConcreteListProcessor(); final TestRunner runner = TestRunners.newTestRunner(proc); final DistributedCache cache = new DistributedCache(); @@ -109,7 +112,32 @@ public class TestAbstractListProcessor { proc.addEntity("name", "id", 1492L); runner.run(); - assertEquals(1, cache.stored.size()); + final Map expectedState = new HashMap<>(); + expectedState.put(AbstractListProcessor.TIMESTAMP, "1492"); + expectedState.put(AbstractListProcessor.IDENTIFIER_PREFIX + ".1", "id"); + runner.getStateManager().assertStateEquals(expectedState, Scope.CLUSTER); + } + + @Test + public void testStateMigrated() throws InitializationException { + final ConcreteListProcessor proc = new ConcreteListProcessor(); + final TestRunner runner = TestRunners.newTestRunner(proc); + final DistributedCache cache = new DistributedCache(); + runner.addControllerService("cache", cache); + runner.enableControllerService(cache); + runner.setProperty(AbstractListProcessor.DISTRIBUTED_CACHE_SERVICE, "cache"); + + final String serviceState = "{\"latestTimestamp\":1492,\"matchingIdentifiers\":[\"id\"]}"; + final String cacheKey = runner.getProcessor().getIdentifier() + ".lastListingTime./path"; + cache.stored.put(cacheKey, serviceState); + + runner.run(); + + final MockStateManager stateManager = runner.getStateManager(); + final Map expectedState = new HashMap<>(); + expectedState.put(AbstractListProcessor.TIMESTAMP, "1492"); + expectedState.put(AbstractListProcessor.IDENTIFIER_PREFIX + ".1", "id"); + stateManager.assertStateEquals(expectedState, Scope.CLUSTER); } @Test @@ -174,7 +202,7 @@ public class TestAbstractListProcessor { @Override protected File getPersistenceFile() { - return new File("target/ListProcessor-local-state.json"); + return new File("target/ListProcessor-local-state-" + UUID.randomUUID().toString() + ".json"); } public void addEntity(final String name, final String identifier, final long timestamp) { diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDetectDuplicate.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDetectDuplicate.java index 12a5cd4ad9..e09d6f07be 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDetectDuplicate.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDetectDuplicate.java @@ -16,16 +16,12 @@ */ package org.apache.nifi.processors.standard; -import java.io.File; import java.io.IOException; -import java.io.OutputStream; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; -import org.apache.commons.lang3.SerializationException; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.controller.AbstractControllerService; import org.apache.nifi.distributed.cache.client.Deserializer; @@ -34,25 +30,21 @@ import org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService import org.apache.nifi.distributed.cache.client.Serializer; import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.state.MockStateManager; import org.apache.nifi.util.MockControllerServiceInitializationContext; import org.apache.nifi.util.MockProcessorLog; import org.apache.nifi.util.TestRunner; import org.apache.nifi.util.TestRunners; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class TestDetectDuplicate { - private static Logger LOGGER; - static { System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info"); System.setProperty("org.slf4j.simpleLogger.showDateTime", "true"); System.setProperty("org.slf4j.simpleLogger.log.nifi.io.nio", "debug"); System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.DetectDuplicate", "debug"); System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.TestDetectDuplicate", "debug"); - LOGGER = LoggerFactory.getLogger(TestDetectDuplicate.class); } @Test @@ -114,7 +106,7 @@ public class TestDetectDuplicate { final DistributedMapCacheClientImpl client = new DistributedMapCacheClientImpl(); final ComponentLog logger = new MockProcessorLog("client", client); - final MockControllerServiceInitializationContext clientInitContext = new MockControllerServiceInitializationContext(client, "client", logger); + final MockControllerServiceInitializationContext clientInitContext = new MockControllerServiceInitializationContext(client, "client", logger, new MockStateManager()); client.initialize(clientInitContext); return client; @@ -154,6 +146,7 @@ public class TestDetectDuplicate { } @Override + @SuppressWarnings("unchecked") public V getAndPutIfAbsent(final K key, final V value, final Serializer keySerializer, final Serializer valueSerializer, final Deserializer valueDeserializer) throws IOException { if (exists) { @@ -183,33 +176,4 @@ public class TestDetectDuplicate { public void put(final K key, final V value, final Serializer keySerializer, final Serializer valueSerializer) throws IOException { } } - - private static class StringSerializer implements Serializer { - - @Override - public void serialize(final String value, final OutputStream output) throws SerializationException, IOException { - output.write(value.getBytes(StandardCharsets.UTF_8)); - } - } - - private static void deleteRecursively(final File dataFile) throws IOException { - if (dataFile == null || !dataFile.exists()) { - return; - } - - final File[] children = dataFile.listFiles(); - for (final File child : children) { - if (child.isDirectory()) { - deleteRecursively(child); - } else { - for (int i = 0; i < 100 && child.exists(); i++) { - child.delete(); - } - - if (child.exists()) { - throw new IOException("Could not delete " + dataFile.getAbsolutePath()); - } - } - } - } } diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGetHTTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGetHTTP.java index 5aec796788..9c1d0830af 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGetHTTP.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGetHTTP.java @@ -16,24 +16,18 @@ */ package org.apache.nifi.processors.standard; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import java.io.File; -import java.io.FileInputStream; import java.util.HashMap; import java.util.Map; -import java.util.Properties; +import org.apache.nifi.components.state.Scope; import org.apache.nifi.flowfile.attributes.CoreAttributes; -import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.reporting.InitializationException; import org.apache.nifi.ssl.SSLContextService; import org.apache.nifi.ssl.StandardSSLContextService; import org.apache.nifi.util.MockFlowFile; -import org.apache.nifi.util.MockProcessContext; -import org.apache.nifi.util.MockProcessorInitializationContext; import org.apache.nifi.util.TestRunner; import org.apache.nifi.util.TestRunners; import org.eclipse.jetty.servlet.ServletHandler; @@ -97,14 +91,14 @@ public class TestGetHTTP { controller.setProperty(GetHTTP.FILENAME, "testFile"); controller.setProperty(GetHTTP.ACCEPT_CONTENT_TYPE, "application/json"); - GetHTTP getHTTPProcessor = (GetHTTP) controller.getProcessor(); - assertEquals("", getHTTPProcessor.entityTagRef.get()); - assertEquals("Thu, 01 Jan 1970 00:00:00 GMT", getHTTPProcessor.lastModifiedRef.get()); + controller.getStateManager().assertStateNotSet(GetHTTP.ETAG, Scope.LOCAL); + controller.getStateManager().assertStateNotSet(GetHTTP.LAST_MODIFIED, Scope.LOCAL); controller.run(2); // verify the lastModified and entityTag are updated - assertFalse("".equals(getHTTPProcessor.entityTagRef.get())); - assertFalse("Thu, 01 Jan 1970 00:00:00 GMT".equals(getHTTPProcessor.lastModifiedRef.get())); + controller.getStateManager().assertStateNotEquals(GetHTTP.ETAG, "", Scope.LOCAL); + controller.getStateManager().assertStateNotEquals(GetHTTP.LAST_MODIFIED, "Thu, 01 Jan 1970 00:00:00 GMT", Scope.LOCAL); + // ran twice, but got one...which is good controller.assertTransferCount(GetHTTP.REL_SUCCESS, 1); @@ -141,18 +135,18 @@ public class TestGetHTTP { controller.run(2); // ran twice, got 1...but should have new cached etag controller.assertTransferCount(GetHTTP.REL_SUCCESS, 1); - assertEquals("1", getHTTPProcessor.entityTagRef.get()); + controller.getStateManager().assertStateEquals(GetHTTP.ETAG, "1", Scope.LOCAL); controller.clearTransferState(); // turn off checking for Etag, turn on checking for lastModified, but change value RESTServiceContentModified.IGNORE_LAST_MODIFIED = false; RESTServiceContentModified.IGNORE_ETAG = true; RESTServiceContentModified.modificationDate = System.currentTimeMillis() / 1000 * 1000 + 5000; - String lastMod = getHTTPProcessor.lastModifiedRef.get(); + String lastMod = controller.getStateManager().getState(Scope.LOCAL).get(GetHTTP.LAST_MODIFIED); controller.run(2); // ran twice, got 1...but should have new cached etag controller.assertTransferCount(GetHTTP.REL_SUCCESS, 1); - assertFalse(lastMod.equals(getHTTPProcessor.lastModifiedRef.get())); + controller.getStateManager().assertStateNotEquals(GetHTTP.LAST_MODIFIED, lastMod, Scope.LOCAL); controller.clearTransferState(); // shutdown web service @@ -161,90 +155,6 @@ public class TestGetHTTP { } } - @Test - public void testPersistEtagLastMod() throws Exception { - // delete the config file - File confDir = new File("conf"); - File[] files = confDir.listFiles(); - for (File file : files) { - assertTrue("Failed to delete " + file.getName(), file.delete()); - } - - // set up web service - ServletHandler handler = new ServletHandler(); - handler.addServletWithMapping(RESTServiceContentModified.class, "/*"); - - // create the service - TestServer server = new TestServer(); - server.addHandler(handler); - - try { - server.startServer(); - - // get the server url - String destination = server.getUrl(); - - // set up NiFi mock controller - controller = TestRunners.newTestRunner(GetHTTP.class); - controller.setProperty(GetHTTP.CONNECTION_TIMEOUT, "5 secs"); - controller.setProperty(GetHTTP.FILENAME, "testFile"); - controller.setProperty(GetHTTP.URL, destination); - controller.setProperty(GetHTTP.ACCEPT_CONTENT_TYPE, "application/json"); - - GetHTTP getHTTPProcessor = (GetHTTP) controller.getProcessor(); - - assertEquals("", getHTTPProcessor.entityTagRef.get()); - assertEquals("Thu, 01 Jan 1970 00:00:00 GMT", getHTTPProcessor.lastModifiedRef.get()); - controller.run(2); - - // verify the lastModified and entityTag are updated - String etag = getHTTPProcessor.entityTagRef.get(); - assertFalse("".equals(etag)); - String lastMod = getHTTPProcessor.lastModifiedRef.get(); - assertFalse("Thu, 01 Jan 1970 00:00:00 GMT".equals(lastMod)); - // ran twice, but got one...which is good - controller.assertTransferCount(GetHTTP.REL_SUCCESS, 1); - controller.clearTransferState(); - - files = confDir.listFiles(); - assertEquals(1, files.length); - File file = files[0]; - assertTrue(file.exists()); - Properties props = new Properties(); - FileInputStream fis = new FileInputStream(file); - props.load(fis); - fis.close(); - assertEquals(etag, props.getProperty(GetHTTP.ETAG)); - assertEquals(lastMod, props.getProperty(GetHTTP.LAST_MODIFIED)); - - ProcessorInitializationContext pic = new MockProcessorInitializationContext(controller.getProcessor(), (MockProcessContext) controller.getProcessContext()); - // init causes read from file - getHTTPProcessor.init(pic); - assertEquals(etag, getHTTPProcessor.entityTagRef.get()); - assertEquals(lastMod, getHTTPProcessor.lastModifiedRef.get()); - controller.run(2); - // ran twice, got none...which is good - controller.assertTransferCount(GetHTTP.REL_SUCCESS, 0); - controller.clearTransferState(); - files = confDir.listFiles(); - assertEquals(1, files.length); - file = files[0]; - assertTrue(file.exists()); - props = new Properties(); - fis = new FileInputStream(file); - props.load(fis); - fis.close(); - assertEquals(etag, props.getProperty(GetHTTP.ETAG)); - assertEquals(lastMod, props.getProperty(GetHTTP.LAST_MODIFIED)); - - getHTTPProcessor.onRemoved(); - assertFalse(file.exists()); - - // shutdown web service - } finally { - server.shutdownServer(); - } - } @Test public final void testUserAgent() throws Exception { diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestRouteOnAttribute.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestRouteOnAttribute.java index 2eac3f2b24..3b70dc460c 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestRouteOnAttribute.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestRouteOnAttribute.java @@ -26,6 +26,7 @@ import java.util.Map; import org.apache.nifi.components.ValidationResult; import org.apache.nifi.processor.Relationship; +import org.apache.nifi.state.MockStateManager; import org.apache.nifi.util.MockFlowFile; import org.apache.nifi.util.MockProcessContext; import org.apache.nifi.util.TestRunner; @@ -38,7 +39,7 @@ public class TestRouteOnAttribute { @Test public void testInvalidOnMisconfiguredProperty() { final RouteOnAttribute proc = new RouteOnAttribute(); - final MockProcessContext ctx = new MockProcessContext(proc); + final MockProcessContext ctx = new MockProcessContext(proc, new MockStateManager()); final ValidationResult validationResult = ctx.setProperty("RouteA", "${a:equals('b')"); // Missing closing brace assertFalse(validationResult.isValid()); } @@ -46,7 +47,7 @@ public class TestRouteOnAttribute { @Test public void testInvalidOnNonBooleanProperty() { final RouteOnAttribute proc = new RouteOnAttribute(); - final MockProcessContext ctx = new MockProcessContext(proc); + final MockProcessContext ctx = new MockProcessContext(proc, new MockStateManager()); final ValidationResult validationResult = ctx.setProperty("RouteA", "${a:length()"); // Should be boolean assertFalse(validationResult.isValid()); } diff --git a/pom.xml b/pom.xml index 2af004d6d3..954733fe00 100644 --- a/pom.xml +++ b/pom.xml @@ -102,7 +102,8 @@ language governing permissions and limitations under the License. --> central - + Maven Repository https://repo1.maven.org/maven2 @@ -578,7 +579,7 @@ language governing permissions and limitations under the License. -->
org.apache.activemq - activemq-broker + activemq-broker 5.12.1 tests @@ -748,6 +749,27 @@ language governing permissions and limitations under the License. --> spark-streaming_2.10 1.3.1 + + org.apache.zookeeper + zookeeper + 3.4.7 + + + + + org.apache.curator + curator-test + 3.0.0 + test + + + org.testng + testng + 6.8.8 + test + + + org.apache.nifi nifi-api @@ -1283,71 +1305,100 @@ language governing permissions and limitations under the License. --> - + - - + + - - + + - + - + - + - - + + - - + + - - + + - + - + - - + + - + - - + + - - - - - - + + + + + + @@ -1424,12 +1475,12 @@ language governing permissions and limitations under the License. --> - + disable-doclint 1.8