diff --git a/nifi-api/src/main/java/org/apache/nifi/action/Operation.java b/nifi-api/src/main/java/org/apache/nifi/action/Operation.java index d1bb43f18f..0d5072c832 100644 --- a/nifi-api/src/main/java/org/apache/nifi/action/Operation.java +++ b/nifi-api/src/main/java/org/apache/nifi/action/Operation.java @@ -33,5 +33,6 @@ public enum Operation { Enable, Disable, Batch, - Purge; + Purge, + ClearState; } diff --git a/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/Stateful.java b/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/Stateful.java new file mode 100644 index 0000000000..de32bd7d74 --- /dev/null +++ b/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/Stateful.java @@ -0,0 +1,54 @@ +/* + * 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.annotation.behavior; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Inherited; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateManager; + +/** + *

+ * Annotation that a Processor, ReportingTask, or Controller Service can use to indicate + * that the component makes use of the {@link StateManager}. This annotation provides the + * user with a description of what information is being stored so that the user is able to + * understand what is shown to them and know what they are clearing should they choose to + * clear the state. Additionally, the UI will not show any state information to users if + * this annotation is not present. + *

+ */ +@Documented +@Target({ElementType.TYPE}) +@Retention(RetentionPolicy.RUNTIME) +@Inherited +public @interface Stateful { + /** + * Provides a description of what information is being stored in the {@link StateManager} + */ + String description(); + + /** + * Indicates the Scope(s) associated with the State that is stored and retrieved. + */ + Scope[] scopes(); +} diff --git a/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnConfigurationRestored.java b/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnConfigurationRestored.java new file mode 100644 index 0000000000..eaa89664b9 --- /dev/null +++ b/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnConfigurationRestored.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.annotation.lifecycle; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Inherited; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + *

+ * Marker Annotation that a Processor, Reporting Task, or Controller Service can use to indicate + * that the method with this Annotation should be invoked whenever the component's configuration + * is restored after a restart of NiFi. + *

+ * + *

+ * Methods with this annotation must take zero arguments. + *

+ * + *

+ * Whenever a new component is added to the flow, this method will be called immediately, since + * there is no configuration to restore (in this case all configuration has already been restored, + * since there is no configuration to restore). + *

+ * + * @since 0.5.0 + */ +@Documented +@Target({ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +@Inherited +public @interface OnConfigurationRestored { + +} diff --git a/nifi-api/src/main/java/org/apache/nifi/components/ConfigurableComponent.java b/nifi-api/src/main/java/org/apache/nifi/components/ConfigurableComponent.java index 8b56f482a6..2e9cb4d356 100644 --- a/nifi-api/src/main/java/org/apache/nifi/components/ConfigurableComponent.java +++ b/nifi-api/src/main/java/org/apache/nifi/components/ConfigurableComponent.java @@ -19,6 +19,8 @@ package org.apache.nifi.components; import java.util.Collection; import java.util.List; +import org.apache.nifi.annotation.lifecycle.OnConfigurationRestored; + public interface ConfigurableComponent { /** @@ -49,11 +51,19 @@ public interface ConfigurableComponent { * necessary lazily evaluate it. Any throwable that escapes this method will * simply be ignored. * + * When NiFi is restarted, this method will be called for each 'dynamic' property that is + * added, as well as for each property that is not set to the default value. I.e., if the + * Properties are modified from the default values. If it is undesirable for your use case + * to react to properties being modified in this situation, you can add the {@link OnConfigurationRestored} + * annotation to a method - this will allow the Processor to know when configuration has + * been restored, so that it can determine whether or not to perform some action in the + * onPropertyModified method. + * * @param descriptor the descriptor for the property being modified * @param oldValue the value that was previously set, or null if no value - * was previously set for this property + * was previously set for this property * @param newValue the new property value or if null indicates the property - * was removed + * was removed */ void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue); 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..dd0d0aa40f --- /dev/null +++ b/nifi-api/src/main/java/org/apache/nifi/components/state/Scope.java @@ -0,0 +1,41 @@ +/* + * 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; + + @Override + public String toString() { + return name(); + } +} 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..7ff8cecaa3 --- /dev/null +++ b/nifi-api/src/main/java/org/apache/nifi/components/state/StateManager.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.components.state; + +import java.io.IOException; +import java.util.Map; + +import org.apache.nifi.annotation.behavior.Stateful; + +/** + *

+ * 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. 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 behave in the same + * manner, as a standalone node could be thought of as a "cluster of 1." + *

+ * + *

+ * This mechanism is designed to allow developers to easily store and retrieve small amounts of state. + * The storage mechanism is implementation-specific, but is typically either stored on a remote system + * or on disk. For that reason, one should consider the cost of storing and retrieving this data, and the + * amount of data should be kept to the minimum required. + *

+ * + *

+ * Any component that wishes to use the StateManager should also use the {@link Stateful} annotation to provide + * a description of what state is being stored and what Scope is used. If this annotation is not present, the UI + * will not expose such information or allow DFMs to clear the state. + *

+ */ +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 the current state for the component + * @throws IOException if unable to communicate with the underlying storage mechanism + */ + 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..4d67164fa4 --- /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..e1e4352721 --- /dev/null +++ b/nifi-api/src/main/java/org/apache/nifi/components/state/StateProvider.java @@ -0,0 +1,133 @@ +/* + * 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(); + + /** + * Provides a listing of {@link Scope}s supported by the StateProvider + * @return the {@link Scope}s supported by the configuration + */ + Scope[] getSupportedScopes(); +} 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/AbstractControllerService.java b/nifi-api/src/main/java/org/apache/nifi/controller/AbstractControllerService.java index cd3b9bdc10..2d9a1f3538 100644 --- a/nifi-api/src/main/java/org/apache/nifi/controller/AbstractControllerService.java +++ b/nifi-api/src/main/java/org/apache/nifi/controller/AbstractControllerService.java @@ -21,6 +21,7 @@ import java.util.Map; import org.apache.nifi.components.AbstractConfigurableComponent; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyValue; +import org.apache.nifi.components.state.StateManager; import org.apache.nifi.controller.annotation.OnConfigured; import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.processor.ProcessorInitializationContext; @@ -32,12 +33,14 @@ public abstract class AbstractControllerService extends AbstractConfigurableComp private ControllerServiceLookup serviceLookup; private volatile ConfigurationContext configContext; private ComponentLog logger; + private StateManager stateManager; @Override public final void initialize(final ControllerServiceInitializationContext context) throws InitializationException { this.identifier = context.getIdentifier(); serviceLookup = context.getControllerServiceLookup(); logger = context.getLogger(); + stateManager = context.getStateManager(); init(context); } @@ -93,4 +96,11 @@ public abstract class AbstractControllerService extends AbstractConfigurableComp protected ComponentLog getLogger() { return logger; } + + /** + * @return the StateManager that can be used to store and retrieve state for this Controller Service + */ + protected StateManager getStateManager() { + return stateManager; + } } 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/AbstractSessionFactoryProcessor.java b/nifi-api/src/main/java/org/apache/nifi/processor/AbstractSessionFactoryProcessor.java index 2695dcddfa..cb9b188626 100644 --- a/nifi-api/src/main/java/org/apache/nifi/processor/AbstractSessionFactoryProcessor.java +++ b/nifi-api/src/main/java/org/apache/nifi/processor/AbstractSessionFactoryProcessor.java @@ -19,6 +19,7 @@ package org.apache.nifi.processor; import java.util.Collections; import java.util.Set; +import org.apache.nifi.annotation.lifecycle.OnConfigurationRestored; import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.annotation.lifecycle.OnUnscheduled; import org.apache.nifi.components.AbstractConfigurableComponent; @@ -47,6 +48,7 @@ public abstract class AbstractSessionFactoryProcessor extends AbstractConfigurab private String identifier; private ProcessorLog logger; private volatile boolean scheduled = false; + private volatile boolean configurationRestored = false; private ControllerServiceLookup serviceLookup; private String description; @@ -104,6 +106,22 @@ public abstract class AbstractSessionFactoryProcessor extends AbstractConfigurab scheduled = false; } + @OnConfigurationRestored + public final void updateConfiguredRestoredTrue() { + configurationRestored = true; + } + + /** + * Returns a boolean indicating whether or not the configuration of the Processor has already been restored. + * See the {@link OnConfigurationRestored} annotation for more information about what it means for the configuration + * to be restored. + * + * @return true if configuration has been restored, false otherwise. + */ + protected boolean isConfigurationRestored() { + return configurationRestored; + } + @Override public final String getIdentifier() { return identifier; 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/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 5268ba32d9..693f742215 100644 --- a/nifi-assembly/pom.xml +++ b/nifi-assembly/pom.xml @@ -303,6 +303,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..c432edf7e3 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,25 @@ 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 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..0dde17dca7 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 @@ -182,6 +182,7 @@ public class EndpointConnectionPool { public Thread newThread(final Runnable r) { final Thread thread = defaultFactory.newThread(r); thread.setName("NiFi Site-to-Site Connection Pool Maintenance"); + thread.setDaemon(true); return thread; } }); @@ -761,7 +762,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> 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". @@ -636,6 +638,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". @@ -660,6 +663,305 @@ additivity="false"> +[[state_management]] +State Management +---------------- + +NiFi provides a mechanism for Processors, Reporting Tasks, Controller Services, and the framework itself to persist state. This +allows a Processor, for example, to resume from the place where it left off after NiFi is restarted. Additionally, it allows for +a Processor to store some piece of information so that the Processor can access that information from all of the different nodes +in the cluster. This allows one node to pick up where another node left off, or to coordinate across all of the nodes in a cluster. + +[[state_providers]] +=== Configuring State Providers +When a component decides to store or retrieve state, it does so by providing a "Scope" - either Node-local or Cluster-wide. The +mechanism that is used to store and retrieve this state is then determined based on this Scope, as well as the configured State +Providers. The _nifi.properties_ file contains three different properties that are relevant to configuring these State Providers. +The first is the `nifi.state.management.configuration.file` property specifies an external XML file that is used for configuring +the local and cluster-wide State Providers. This XML file may contain configurations for multiple providers, so the +`nifi.state.management.provider.local` property provides the identifier of the local State Provider configured in this XML file. +Similarly, the `nifi.state.management.provider.cluster` property provides the identifier of the cluster-wide State Provider +configured in this XML file. + +This XML file consists of a top-level `state-management` element, which has one or more `local-provider` and zero or more +`cluster-provider` elements. Each of these elements then contains an `id` element that is used to specify the identifier that can +be referenced in the _nifi.properties_ file, as well as a `class` element that specifies the fully-qualified class name to use +in order to instantiate the State Provider. Finally, each of these elements may have zero or more `property` elements. Each +`property` element has an attribute, `name` that is the name of the property that the State Provider supports. The textual content +of the `property` element is the value of the property. + +Once these State Providers have been configured in the _state-management.xml_ file (or whatever file is configured), those Providers +may be referenced by their identifiers. By default, the Local State Provider is configured to be a `WriteAheadLocalStateProvider` that +persists the data to the _$NIFI_HOME/state_ directory. The default Cluster State Provider is configured to be a `ZooKeeperStateProvider`. +The default ZooKeeper-based provider must have its `Connect String` property populated before it can be used. It is also advisable, +if multiple NiFi instances will use the same ZooKeeper instance, that the value of the `Root Node` property be changed. For instance, +one might set the value to `/nifi//production`. A `Connect String` takes the form of comma separated : tuples, +such as my-zk-server1:2181,my-zk-server2:2181,my-zk-server3:2181. In the event a port is not specified for any of the hosts, the ZooKeeper +default of 2181 is assumed. + +When adding data to ZooKeeper, there are two options for Access Control: `Open` and `CreatorOnly`. If the `Access Control` property is +set to `Open`, then anyone is allowed to log into ZooKeeper and have full permissions to see, change, delete, or administer the data. +If `CreatorOnly` is specified, then only the user that created the data is allowed to read, change, delete, or administer the data. +In order to use the `CreatorOnly` option, NiFi must provide some form of authentication. See the <> +section below for more information on how to configure authentication. + +If NiFi is configured to run in a standalone mode, the `cluster-provider` element need not be populated in the _state-management.xml_ +file and will actually be ignored if they are populated. However, the `local-provider` element must always be present and populated. +Additionally, if NiFi is run in a cluster, each node must also have the `cluster-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-management.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. + +It should be noted that if Processors and other components save state using the Clustered scope, the Local State Provider will be used +if the instance is a standalone instance (not in a cluster) or is disconnected from the cluster. This also means that if a standalone instance +is migrated to become a cluster, then that state will no longer be available, as the component will begin using the Clustered State Provider +instead of the Local State Provider. + + +[[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 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. The servers are specified as properties in the form of `server.1`, `server.2`, to `server.n`. 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. Additionally, the port to +listen on for client connections must be opened in the firewall. The default value for this is _2181_ but can be configured via the _clientPort_ property +in the _zookeeper.properties_ file. + +When using an embedded ZooKeeper, the _conf/zookeeper.properties_ file has a property named `dataDir`. By default, this value is set to `./state/zookeeper`. +If more than one NiFi node is running an embedded ZooKeeper, it is important to tell the server which one it is. This is accomplished by creating a file named +_myid_ and placing it in ZooKeeper's data directory. The contents of this file should be the index of the server as specific by the `server.`. So for +one of the ZooKeeper servers, we will accomplish this by performing the following commands: + +[source] +cd $NIFI_HOME +mkdir state +mkdir state/zookeeper +echo 1 > state/zookeeper/myid + +For the next NiFi Node that will run ZooKeeper, we can accomplish this by performing the following commands: + +[source] +cd $NIFI_HOME +mkdir state +mkdir state/zookeeper +echo 2 > state/zookeeper/myid + +And so on. + +For more information on the properties used to administer ZooKeeper, see the +link:https://zookeeper.apache.org/doc/current/zookeeperAdmin.html[ZooKeeper Admin Guide]. + +For information on securing the embedded ZooKeeper Server, see the <> section below. + + + +[[zk_access_control]] +=== ZooKeeper Access Control +ZooKeeper provides Access Control to its data via an Access Control List (ACL) mechanism. When data is written to ZooKeeper, NiFi will provide an ACL +that indicates that any user is allowed to have full permissions to the data, or an ACL that indicates that only the user that created the data is +allowed to access the data. Which ACL is used depends on the value of the `Access Control` property for the `ZooKeeperStateProvider` (see the +<> section for more information). + +In order to use an ACL that indicates that only the Creator is allowed to access the data, we need to tell ZooKeeper who the Creator is. There are two +mechanisms for accomplishing this. The first mechanism is to provide authentication using Kerberos. See <> for more information. + +The second option is to use a user name and password. This is configured by specifying a value for the `Username` and a value for the `Password` properties +for the `ZooKeeperStateProvider` (see the <> section for more information). The important thing to keep in mind here, though, is that ZooKeeper +will pass around the password in plain text. This means that using a user name and password should not be used unless ZooKeeper is running on localhost as a +one-instance cluster, or if communications with ZooKeeper occur only over encrypted communications, such as a VPN or an SSL connection. ZooKeeper will be +providing support for SSL connections in version 3.5.0. + + + +[[securing_zookeeper]] +=== Securing ZooKeeper + +When NiFi communicates with ZooKeeper, all communications, by default, are non-secure, and anyone who logs into ZooKeeper is able to view and manipulate all +of the NiFi state that is stored in ZooKeeper. To prevent this, we can use Kerberos to manage the authentication. At this time, ZooKeeper does not provide +support for encryption via SSL. Support for SSL in ZooKeeper is being actively developed and is expected to be available in the 3.5.x release version. + +In order to secure the communications, we need to ensure that both the client and the server support the same configuration. Instructions for configuring the +NiFi ZooKeeper client and embedded ZooKeeper server to use Kerberos are provided below. + + +[[zk_kerberos_client]] +==== Kerberizing NiFi's ZooKeeper Client +The preferred mechanism for authenticating users with ZooKeeper is to use Kerberos. In order to use Kerberos to authenticate, we must configure a few +system properties, so that the ZooKeeper client knows who the user is and where the KeyTab file is. All nodes configured to store cluster-wide state +using `ZooKeeperStateProvider` and using Kerberos should follow these steps. + +First, we must create the Principal that we will use when communicating with ZooKeeper. This is generally done via the `kadmin` tool: + +[source] +kadmin: addprinc "nifi@EXAMPLE.COM" + +A Kerberos Principal is made up of three parts: the primary, the instance, and the realm. Here, we are creating a Principal with the primary `nifi`, +no instance, and the realm `EXAMPLE.COM`. The primary (`nifi`, in this case) is the identifier that will be used to identify the user when authenticating +via Kerberos. + +After we have created our Principal, we will need to create a KeyTab for the Principal: + +[source] +kadmin: xst -k nifi.keytab nifi@EXAMPLE.COM + +This will create a file in the current directory named `nifi.keytab`. We can now copy that file into the _$NIFI_HOME/conf/_ directory. We should ensure +that only the user that will be running NiFi is allowed to read this file. + +Next, we need to configure NiFi to use this KeyTab for authentication. Since ZooKeeper uses the Java Authentication and Authorization Service (JAAS), we need to +create a JAAS-compatible file. In the `$NIFI_HOME/conf/` directory, create a file named `zookeeper-jaas.conf` and add to it the following snippet: + +[source] +Client { + com.sun.security.auth.module.Krb5LoginModule required + useKeyTab=true + keyTab="./conf/nifi.keytab" + storeKey=true + useTicketCache=false + principal="nifi@EXAMPLE.COM"; +}; + + +Finally, we need to tell NiFi to use this as our JAAS configuration. This is done by setting a JVM System Property, so we will edit the _conf/bootstrap.conf_ file. +We add the following line anywhere in this file in order to tell the NiFi JVM to use this configuration: + +[source] +java.arg.15=-Djava.security.auth.login.config=./conf/zookeeper-jaas.conf + +We can initialize our Kerberos ticket by running the following command: + +[source] +kinit nifi + +Note, the above `kinit` command requires that Kerberos client libraries be installed. This is accomplished in Fedora-based Linux distributions via: + +[source] +yum install krb5-workstation krb5-libs krb5-auth-dialog + +Once this is complete, the /etc/krb5.conf will need to be configured appropriately for your organization's Kerberos envrionment. + +Now, when we start NiFi, it will use Kerberos to authentication as the `nifi` user when communicating with ZooKeeper. + + + +[[zk_kerberos_server]] +==== Kerberizing Embedded ZooKeeper Server +When using the embedded ZooKeeper server, we may choose to secure the server by using Kerberos. All nodes configured to launch an embedded ZooKeeper +and using Kerberos should follow these steps. + +If Kerberos is not already setup in your environment, you can find information on installing and setting up a Kerberos Server at +link:https://access.redhat.com/documentation/en-US/Red_Hat_Enterprise_Linux/6/html/Managing_Smart_Cards/Configuring_a_Kerberos_5_Server.html[https://access.redhat.com/documentation/en-US/Red_Hat_Enterprise_Linux/6/html/Managing_Smart_Cards/Configuring_a_Kerberos_5_Server.html] +. This guide assumes that Kerberos already has been installed in the environment in which NiFi is running. + +In order to use Kerberos, we first need to generate a Kerberos Principal for our ZooKeeper server. This is accomplished via the `kadmin` tool: + +[source] +kadmin: addprinc "zookeeper/myHost.example.com@EXAMPLE.COM" + +Here, we are creating a Principal with the primary `zookeeper/myHost.example.com`, using the realm `EXAMPLE.COM`. We need to use a Principal whose +name is `/`. In this case, the service is `zookeeper` and the instance name is `myHost.example.com` (the fully qualified name of our host). + +Next, we will need to create a KeyTab for this Principal: + +[source] +kadmin: xst -k zookeeper-server.keytab zookeeper/myHost.example.com@EXAMPLE.COM + +This will create a file in the current directory named `zookeeper-server.keytab`. We can now copy that file into the `$NIFI_HOME/conf/` directory. We should ensure +that only the user that will be running NiFi is allowed to read this file. + +We will need to repeat the above steps for each of the instances of NiFi that will be running the embedded ZooKeeper server, being sure to replace _myHost.example.com_ with +_myHost2.example.com_, or whatever fully qualified hostname the ZooKeeper server will be run on. + +Now that we have our KeyTab for each of the servers that will be running NiFi, we will need to configure NiFi's embedded ZooKeeper server to use this configuration. +ZooKeeper uses the Java Authentication and Authorization Service (JAAS), so we need to create a JAAS-compatible file In the `$NIFI_HOME/conf/` directory, create a file +named `zookeeper-jaas.conf` (this file will already exist if the Client has already been configured to authenticate via Kerberos. That's okay, just add to the file). +We will add to this file, the following snippet: + +[source] +Server { + com.sun.security.auth.module.Krb5LoginModule required + useKeyTab=true + keyTab="./conf/zookeeper-server.keytab" + storeKey=true + useTicketCache=false + principal="zookeeper/myHost.example.com@EXAMPLE.COM"; +}; + +Be sure to replace the value of _principal_ above with the appropriate Principal, including the fully qualified domain name of the server. + +Next, we need to tell NiFi to use this as our JAAS configuration. This is done by setting a JVM System Property, so we will edit the `conf/bootstrap.conf` file. +If the Client has already been configured to use Kerberos, this is not necessary, as it was done above. Otherwise, we will add the following line to our _bootstrap.conf_ file: + +[source] +java.arg.15=-Djava.security.auth.login.config=./conf/zookeeper-jaas.conf + +We will want to initialize our Kerberos ticket by running the following command: + +[source] +kinit "zookeeper/myHost.example.com@EXAMPLE.COM" + +Again, be sure to replace the Principal with the appropriate value, including your realm and your fully qualified hostname. + +Finally, we need to tell the Kerberos server to use the SASL Authentication Provider. To do this, we edit the `$NIFI_HOME/conf/zookeeper.properties` file and add the following +lines: + +[source] +authProvider.1=org.apache.zookeeper.server.auth.SASLAuthenticationProvider +jaasLoginRenew=3600000 +requireClientAuthScheme=sasl + +The last line is optional but specifies that clients MUST use Kerberos to communicate with our ZooKeeper instance. + +Now, we can start NiFi, and the embedded ZooKeeper server will use Kerberos as the authentication mechanism. + + + + +[[troubleshooting_kerberos]] +==== Troubleshooting Kerberos Configuration +When using Kerberos, it is import to use fully-qualified domain names and not use _localhost_. Please ensure that the fully qualified hostname of each server is used +in the following locations: + + - _conf/zookeeper.properties_ file should use FQDN for `server.1`, `server.2`, ..., `server.N` values. + - The `Connect String` property of the ZooKeeperStateProvider + - The /etc/hosts file should also resolve the FQDN to an IP address that is *not* _127.0.0.1_. + +Failure to do so, may result in errors similar to the following: + +[source] +2016-01-08 16:08:57,888 ERROR [pool-26-thread-1-SendThread(localhost:2181)] o.a.zookeeper.client.ZooKeeperSaslClient An error: (java.security.PrivilegedActionException: javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Server not found in Kerberos database (7) - LOOKING_UP_SERVER)]) occurred when evaluating Zookeeper Quorum Member's received SASL token. Zookeeper Client will go to AUTH_FAILED state. + +If there are problems communicating or authenticating with Kerberos, +link:http://docs.oracle.com/javase/7/docs/technotes/guides/security/jgss/tutorials/Troubleshooting.html[this Troubleshooting Guide] may be of value. + +One of the most important notes in the above Troubleshooting guide is the mechanism for turning on Debug output for Kerberos. +This is done by setting the `sun.security.krb5.debug` environment variable. +In NiFi, this is accomplished by adding the following line to the _$NIFI_HOME/conf/bootstrap.conf` file: + +[source] +java.arg.16=-Dsun.security.krb5.debug=true + +This will cause the debug output to be written to the NiFi Bootstrap log file. By default, this is located at _$NIFI_HOME/logs/nifi-bootstrap.log_. +This output can be rather verbose but provides extremely valuable information for troubleshooting Kerberos failures. + + + [[bootstrap_properties]] Bootstrap Properties -------------------- @@ -824,6 +1126,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..75f9ecf99e 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) and scope. 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 +Processors' 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 @@ -728,7 +793,7 @@ public final class InvokeHTTP extends AbstractProcessor { ---- === Documenting Related Components -Often Processors and ControllerServices are related to one another. Sometimes its a put/get relation as in `PutFile` and `GetFile`. +Often Processors and ControllerServices are related to one another. Sometimes it is a put/get relation as in `PutFile` and `GetFile`. Sometimes a Processor uses a ControllerService like `InvokeHTTP` and `StandardSSLContextService`. Sometimes one ControllerService uses another like `DistributedMapCacheClientService` and `DistributedMapCacheServer`. Developers of these extension points may relate these different components using the `SeeAlso` tag. This annotation links these components in the documentation. 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..81ad988ca9 --- /dev/null +++ b/nifi-mock/src/main/java/org/apache/nifi/state/MockStateManager.java @@ -0,0 +1,278 @@ +/* + * 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.io.IOException; +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.nifi.annotation.behavior.Stateful; +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); + + private volatile boolean failToGetLocalState = false; + private volatile boolean failToSetLocalState = false; + private volatile boolean failToGetClusterState = false; + private volatile boolean failToSetClusterState = false; + + private final boolean usesLocalState; + private final boolean usesClusterState; + + public MockStateManager(final Object component) { + final Stateful stateful = component.getClass().getAnnotation(Stateful.class); + if (stateful == null) { + usesLocalState = false; + usesClusterState = false; + } else { + final Scope[] scopes = stateful.scopes(); + boolean local = false; + boolean cluster = false; + + for (final Scope scope : scopes) { + if (scope == Scope.LOCAL) { + local = true; + } else if (scope == Scope.CLUSTER) { + cluster = true; + } + } + + usesLocalState = local; + usesClusterState = cluster; + } + } + + @Override + public synchronized void setState(final Map state, final Scope scope) throws IOException { + verifyAnnotation(scope); + verifyCanSet(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) throws IOException { + verifyAnnotation(scope); + verifyCanGet(scope); + return retrieveState(scope); + } + + private synchronized StateMap retrieveState(final Scope scope) { + verifyAnnotation(scope); + if (scope == Scope.CLUSTER) { + return clusterStateMap; + } else { + return localStateMap; + } + } + + @Override + public synchronized boolean replace(final StateMap oldValue, final Map newValue, final Scope scope) throws IOException { + verifyAnnotation(scope); + if (scope == Scope.CLUSTER) { + if (oldValue == clusterStateMap) { + verifyCanSet(scope); + clusterStateMap = new MockStateMap(newValue, versionIndex.incrementAndGet()); + return true; + } + + return false; + } else { + if (oldValue == localStateMap) { + verifyCanSet(scope); + localStateMap = new MockStateMap(newValue, versionIndex.incrementAndGet()); + return true; + } + + return false; + } + } + + @Override + public synchronized void clear(final Scope scope) throws IOException { + verifyAnnotation(scope); + setState(Collections. emptyMap(), scope); + } + + private void verifyCanSet(final Scope scope) throws IOException { + final boolean failToSet = (scope == Scope.LOCAL) ? failToSetLocalState : failToSetClusterState; + if (failToSet) { + throw new IOException("Unit Test configured to throw IOException if " + scope + " State is set"); + } + } + + private void verifyCanGet(final Scope scope) throws IOException { + final boolean failToGet = (scope == Scope.LOCAL) ? failToGetLocalState : failToGetClusterState; + if (failToGet) { + throw new IOException("Unit Test configured to throw IOException if " + scope + " State is retrieved"); + } + } + + private void verifyAnnotation(final Scope scope) { + // ensure that the @Stateful annotation is present with the appropriate Scope + if ((scope == Scope.LOCAL && !usesLocalState) || (scope == Scope.CLUSTER && !usesClusterState)) { + Assert.fail("Component is attempting to set or retrieve state with a scope of " + scope + " but does not declare that it will use " + + scope + " state. A @Stateful annotation should be added to the component with a scope of " + scope); + } + } + + private String getValue(final String key, final Scope scope) { + final StateMap stateMap; + if (scope == Scope.CLUSTER) { + stateMap = clusterStateMap; + } else { + stateMap = localStateMap; + } + + 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 = retrieveState(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 = retrieveState(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.assertNotSame("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.assertEquals("Expected state not to be set for Scope " + scope + ", but it was set", -1L, stateMap.getVersion()); + } + + /** + * Specifies whether or not the State Manager should throw an IOException when state is set for the given scope. + * Note that calls to {@link #replace(StateMap, Map, Scope)} will fail only if the state would be set (i.e., if + * we call replace and the StateMap does not match the old value, it will not fail). + * + * Also note that if setting state is set to fail, clearing will also fail, as clearing is thought of as setting the + * state to empty + * + * @param scope the scope that should (or should not) fail + * @param fail whether or not setting state should fail + */ + public void setFailOnStateSet(final Scope scope, final boolean fail) { + if (scope == Scope.LOCAL) { + failToSetLocalState = fail; + } else { + failToSetClusterState = fail; + } + } + + /** + * Specifies whether or not the State Manager should throw an IOException when state is retrieved for the given scope. + * + * @param scope the scope that should (or should not) fail + * @param fail whether or not retrieving state should fail + */ + public void setFailOnStateGet(final Scope scope, final boolean fail) { + if (scope == Scope.LOCAL) { + failToGetLocalState = fail; + } else { + failToGetClusterState = fail; + } + } +} 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..754bec0baa 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(controllerService)); } - 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..02a1d8aa42 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(component)); + } + /** * 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 bf5977cb35..7358b42137 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 @@ -46,6 +46,7 @@ import java.util.concurrent.atomic.AtomicLong; import org.apache.nifi.annotation.behavior.TriggerSerially; import org.apache.nifi.annotation.lifecycle.OnAdded; +import org.apache.nifi.annotation.lifecycle.OnConfigurationRestored; import org.apache.nifi.annotation.lifecycle.OnDisabled; import org.apache.nifi.annotation.lifecycle.OnEnabled; import org.apache.nifi.annotation.lifecycle.OnRemoved; @@ -57,6 +58,7 @@ import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.state.StateManager; import org.apache.nifi.controller.ConfigurationContext; import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.queue.QueueSize; @@ -68,6 +70,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 { @@ -79,6 +82,8 @@ public class StandardProcessorTestRunner implements TestRunner { private final SharedSessionState sharedState; private final AtomicLong idGenerator; private final boolean triggerSerially; + private final MockStateManager processorStateManager; + private final Map controllerServiceStateManagers = new HashMap<>(); private int numThreads = 1; private final AtomicInteger invocations = new AtomicInteger(0); @@ -101,7 +106,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.processorStateManager = new MockStateManager(processor); + this.context = new MockProcessContext(processor, processorStateManager); detectDeprecatedAnnotations(processor); @@ -116,6 +122,8 @@ public class StandardProcessorTestRunner implements TestRunner { } triggerSerially = null != processor.getClass().getAnnotation(TriggerSerially.class); + + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, processor); } @SuppressWarnings("deprecation") @@ -578,7 +586,9 @@ public class StandardProcessorTestRunner implements TestRunner { final MockProcessorLog logger = new MockProcessorLog(identifier, service); controllerServiceLoggers.put(identifier, logger); - final MockControllerServiceInitializationContext initContext = new MockControllerServiceInitializationContext(requireNonNull(service), requireNonNull(identifier), logger); + final MockStateManager serviceStateManager = new MockStateManager(service); + final MockControllerServiceInitializationContext initContext = new MockControllerServiceInitializationContext(requireNonNull(service), requireNonNull(identifier), logger, serviceStateManager); + controllerServiceStateManagers.put(identifier, serviceStateManager); initContext.addControllerServices(context); service.initialize(initContext); @@ -598,7 +608,12 @@ public class StandardProcessorTestRunner implements TestRunner { @Override public void assertNotValid(final ControllerService service) { - final ValidationContext validationContext = new MockValidationContext(context).getControllerServiceValidationContext(service); + final StateManager serviceStateManager = controllerServiceStateManagers.get(service.getIdentifier()); + if (serviceStateManager == null) { + throw new IllegalStateException("Controller Service has not been added to this TestRunner via the #addControllerService method"); + } + + final ValidationContext validationContext = new MockValidationContext(context, serviceStateManager).getControllerServiceValidationContext(service); final Collection results = context.getControllerService(service.getIdentifier()).validate(validationContext); for (final ValidationResult result : results) { @@ -612,7 +627,12 @@ public class StandardProcessorTestRunner implements TestRunner { @Override public void assertValid(final ControllerService service) { - final ValidationContext validationContext = new MockValidationContext(context).getControllerServiceValidationContext(service); + final StateManager serviceStateManager = controllerServiceStateManagers.get(service.getIdentifier()); + if (serviceStateManager == null) { + throw new IllegalStateException("Controller Service has not been added to this TestRunner via the #addControllerService method"); + } + + final ValidationContext validationContext = new MockValidationContext(context, serviceStateManager).getControllerServiceValidationContext(service); final Collection results = context.getControllerService(service.getIdentifier()).validate(validationContext); for (final ValidationResult result : results) { @@ -718,11 +738,16 @@ public class StandardProcessorTestRunner implements TestRunner { @Override public ValidationResult setProperty(final ControllerService service, final PropertyDescriptor property, final String value) { + final MockStateManager serviceStateManager = controllerServiceStateManagers.get(service.getIdentifier()); + if (serviceStateManager == null) { + throw new IllegalStateException("Controller service " + service + " has not been added to this TestRunner via the #addControllerService method"); + } + final ControllerServiceConfiguration configuration = getConfigToUpdate(service); 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, serviceStateManager).getControllerServiceValidationContext(service); final ValidationResult validationResult = property.validate(value, validationContext); updatedProps.put(property, value); @@ -771,6 +796,22 @@ public class StandardProcessorTestRunner implements TestRunner { sharedState.clearProvenanceEvents(); } + @Override + public MockStateManager getStateManager() { + return processorStateManager; + } + + /** + * Returns the State Manager for the given Controller Service. + * + * @param controllerService the Controller Service whose State Manager should be returned + * @return the State Manager for the given Controller Service + */ + @Override + public MockStateManager getStateManager(final ControllerService controllerService) { + return controllerServiceStateManagers.get(controllerService.getIdentifier()); + } + public MockProcessorLog getLogger() { return logger; } 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 2a4afd9a11..5e45299ec8 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 { @@ -838,4 +839,15 @@ public interface TestRunner { * @return the logger */ public MockProcessorLog getControllerServiceLogger(final String identifier); + + /** + * @return the State Manager that is used to stored and retrieve state + */ + MockStateManager getStateManager(); + + /** + * @param service the controller service of interest + * @return the State Manager that is used to store and retrieve state for the given controller service + */ + MockStateManager getStateManager(ControllerService service); } 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-client-dto/src/main/java/org/apache/nifi/web/api/dto/ComponentStateDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ComponentStateDTO.java new file mode 100644 index 0000000000..9036d53776 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ComponentStateDTO.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.web.api.dto; + +import com.wordnik.swagger.annotations.ApiModelProperty; + +import javax.xml.bind.annotation.XmlType; + +/** + * State for a given component. + */ +@XmlType(name = "componentState") +public class ComponentStateDTO { + + private String componentId; + private String stateDescription; + private StateMapDTO clusterState; + private StateMapDTO localState; + + /** + * @return The component identifier + */ + @ApiModelProperty( + value = "The component identifier." + ) + public String getComponentId() { + return componentId; + } + + public void setComponentId(String componentId) { + this.componentId = componentId; + } + + /** + * @return Description of the state this component persists. + */ + @ApiModelProperty( + value = "Description of the state this component persists." + ) + public String getStateDescription() { + return stateDescription; + } + + public void setStateDescription(String stateDescription) { + this.stateDescription = stateDescription; + } + + /** + * @return The cluster state for this component, or null if this NiFi is a standalone instance + */ + @ApiModelProperty( + value = "The cluster state for this component, or null if this NiFi is a standalone instance." + ) + public StateMapDTO getClusterState() { + return clusterState; + } + + public void setClusterState(StateMapDTO clusterState) { + this.clusterState = clusterState; + } + + /** + * @return The local state for this component + */ + @ApiModelProperty( + value = "The local state for this component." + ) + public StateMapDTO getLocalState() { + return localState; + } + + public void setLocalState(StateMapDTO localState) { + this.localState = localState; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerServiceDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerServiceDTO.java index 659be91389..5d51698375 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerServiceDTO.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerServiceDTO.java @@ -33,6 +33,7 @@ public class ControllerServiceDTO extends NiFiComponentDTO { private String comments; private String availability; private String state; + private Boolean persistsState; private Map properties; private Map descriptors; @@ -101,6 +102,20 @@ public class ControllerServiceDTO extends NiFiComponentDTO { this.availability = availability; } + /** + * @return whether this controller service persists state + */ + @ApiModelProperty( + value = "Whether the controller service persists state." + ) + public Boolean getPersistsState() { + return persistsState; + } + + public void setPersistsState(Boolean persistsState) { + this.persistsState = persistsState; + } + /** * @return The state of this controller service. Possible values are ENABLED, ENABLING, DISABLED, DISABLING */ diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorDTO.java index 0e4ddde3f7..b0b9daae59 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorDTO.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorDTO.java @@ -37,6 +37,7 @@ public class ProcessorDTO extends NiFiComponentDTO { private Boolean supportsParallelProcessing; private Boolean supportsEventDriven; private Boolean supportsBatching; + private Boolean persistsState; private String inputRequirement; private ProcessorConfigDTO config; @@ -122,6 +123,20 @@ public class ProcessorDTO extends NiFiComponentDTO { this.supportsParallelProcessing = supportsParallelProcessing; } + /** + * @return whether this processor persists state + */ + @ApiModelProperty( + value = "Whether the processor persists state." + ) + public Boolean getPersistsState() { + return persistsState; + } + + public void setPersistsState(Boolean persistsState) { + this.persistsState = persistsState; + } + /** * @return the input requirement of this processor */ diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ReportingTaskDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ReportingTaskDTO.java index b8268293f7..182535d915 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ReportingTaskDTO.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ReportingTaskDTO.java @@ -33,6 +33,7 @@ public class ReportingTaskDTO extends NiFiComponentDTO { private String state; private String availability; private String comments; + private Boolean persistsState; private String schedulingPeriod; private String schedulingStrategy; @@ -105,6 +106,20 @@ public class ReportingTaskDTO extends NiFiComponentDTO { this.schedulingPeriod = schedulingPeriod; } + /** + * @return whether this reporting task persists state + */ + @ApiModelProperty( + value = "Whether the reporting task persists state." + ) + public Boolean getPersistsState() { + return persistsState; + } + + public void setPersistsState(Boolean persistsState) { + this.persistsState = persistsState; + } + /** * @return current scheduling state of the reporting task */ diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/StateEntryDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/StateEntryDTO.java new file mode 100644 index 0000000000..93d9c3d2ac --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/StateEntryDTO.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.web.api.dto; + +import com.wordnik.swagger.annotations.ApiModelProperty; + +import javax.xml.bind.annotation.XmlType; + +/** + * Mapping of state for a given scope. + */ +@XmlType(name = "stateMap") +public class StateEntryDTO { + + private String key; + private String value; + + private String clusterNodeId; // include when clustered and scope is local + private String clusterNodeAddress; // include when clustered and scope is local + + /** + * @return the key for this state + */ + @ApiModelProperty( + value = "The key for this state." + ) + public String getKey() { + return key; + } + + public void setKey(String key) { + this.key = key; + } + + /** + * @return the value for this state + */ + @ApiModelProperty( + value = "The value for this state." + ) + public String getValue() { + return value; + } + + public void setValue(String value) { + this.value = value; + } + + /** + * @return identifier of the node where this state originated + */ + @ApiModelProperty( + value = "The identifier for the node where the state originated." + ) + public String getClusterNodeId() { + return clusterNodeId; + } + + public void setClusterNodeId(String clusterNodeId) { + this.clusterNodeId = clusterNodeId; + } + + /** + * @return label to use to show which node this state originated from + */ + @ApiModelProperty( + value = "The label for the node where the state originated." + ) + public String getClusterNodeAddress() { + return clusterNodeAddress; + } + + public void setClusterNodeAddress(String clusterNodeAddress) { + this.clusterNodeAddress = clusterNodeAddress; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/StateMapDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/StateMapDTO.java new file mode 100644 index 0000000000..be5e80868d --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/StateMapDTO.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.web.api.dto; + +import com.wordnik.swagger.annotations.ApiModelProperty; + +import javax.xml.bind.annotation.XmlType; +import java.util.List; + +/** + * Mapping of state for a given scope. + */ +@XmlType(name = "stateMap") +public class StateMapDTO { + + private String scope; + private List state; + + /** + * @return The scope of this StateMap + */ + @ApiModelProperty( + value = "The scope of this StateMap." + ) + public String getScope() { + return scope; + } + + public void setScope(String scope) { + this.scope = scope; + } + + /** + * @return The state + */ + @ApiModelProperty( + value = "The state." + ) + public List getState() { + return state; + } + + public void setState(List state) { + this.state = state; + } + +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ComponentStateEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ComponentStateEntity.java new file mode 100644 index 0000000000..dfb475c662 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ComponentStateEntity.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.web.api.entity; + +import org.apache.nifi.web.api.dto.ComponentStateDTO; + +import javax.xml.bind.annotation.XmlRootElement; + +/** + * A serialized representation of this class can be placed in the entity body of a request or response to or from the API. This particular entity holds a reference to an ComponentStateDTO. + */ +@XmlRootElement(name = "componentStateEntity") +public class ComponentStateEntity extends Entity { + + private ComponentStateDTO componentState; + + /** + * The ComponentStateDTO that is being serialized. + * + * @return The ComponentStateDTO object + */ + public ComponentStateDTO getComponentState() { + return componentState; + } + + public void setComponentState(ComponentStateDTO componentState) { + this.componentState = componentState; + } + +} 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..cb4dd6ac62 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 @@ -33,7 +33,6 @@ import org.apache.commons.lang3.StringUtils; * @Threadsafe */ public class NodeIdentifier { - /** * the unique identifier for the node */ @@ -61,13 +60,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 +95,9 @@ public class NodeIdentifier { validatePort(apiPort); validatePort(socketPort); + if (siteToSitePort != null) { + validatePort(siteToSitePort); + } this.id = id; this.apiAddress = apiAddress; @@ -86,6 +105,9 @@ public class NodeIdentifier { this.socketAddress = socketAddress; this.socketPort = socketPort; this.nodeDn = dn; + this.siteToSiteAddress = siteToSiteAddress == null ? apiAddress : siteToSiteAddress; + this.siteToSitePort = siteToSitePort; + this.siteToSiteSecure = siteToSiteSecure; } public String getId() { @@ -118,6 +140,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 +200,7 @@ public class NodeIdentifier { if (this.socketPort != other.socketPort) { return false; } + return true; } @@ -177,7 +213,9 @@ public class NodeIdentifier { @Override public String toString() { - return "[" + "id=" + id + ", apiAddress=" + apiAddress + ", apiPort=" + apiPort + ", socketAddress=" + socketAddress + ", socketPort=" + socketPort + ']'; + return "[" + "id=" + id + ", apiAddress=" + apiAddress + ", apiPort=" + apiPort + + ", socketAddress=" + socketAddress + ", socketPort=" + socketPort + + ", siteToSiteAddress=" + siteToSiteAddress + ", siteToSitePort=" + siteToSitePort + ']'; } } 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 6c3764448d..a8f6118e1d 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 @@ -71,6 +71,7 @@ import javax.xml.transform.stream.StreamResult; import org.apache.commons.lang3.StringUtils; import org.apache.nifi.admin.service.AuditService; import org.apache.nifi.annotation.lifecycle.OnAdded; +import org.apache.nifi.annotation.lifecycle.OnConfigurationRestored; import org.apache.nifi.annotation.lifecycle.OnRemoved; import org.apache.nifi.cluster.BulletinsPayload; import org.apache.nifi.cluster.HeartbeatPayload; @@ -87,6 +88,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 +128,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.Heartbeater; import org.apache.nifi.controller.ReportingTaskNode; @@ -148,6 +151,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; @@ -197,6 +201,7 @@ import org.apache.nifi.util.ReflectionUtils; import org.apache.nifi.web.OptimisticLockingManager; import org.apache.nifi.web.Revision; import org.apache.nifi.web.UpdateRevision; +import org.apache.nifi.web.api.dto.ComponentStateDTO; import org.apache.nifi.web.api.dto.ControllerServiceDTO; import org.apache.nifi.web.api.dto.ControllerServiceReferencingComponentDTO; import org.apache.nifi.web.api.dto.DropRequestDTO; @@ -211,6 +216,8 @@ import org.apache.nifi.web.api.dto.RemoteProcessGroupContentsDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO; import org.apache.nifi.web.api.dto.ReportingTaskDTO; +import org.apache.nifi.web.api.dto.StateEntryDTO; +import org.apache.nifi.web.api.dto.StateMapDTO; import org.apache.nifi.web.api.dto.provenance.ProvenanceDTO; import org.apache.nifi.web.api.dto.provenance.ProvenanceEventDTO; import org.apache.nifi.web.api.dto.provenance.ProvenanceRequestDTO; @@ -219,6 +226,7 @@ import org.apache.nifi.web.api.dto.status.ClusterStatusHistoryDTO; import org.apache.nifi.web.api.dto.status.NodeStatusHistoryDTO; import org.apache.nifi.web.api.dto.status.StatusHistoryDTO; import org.apache.nifi.web.api.dto.status.StatusSnapshotDTO; +import org.apache.nifi.web.api.entity.ComponentStateEntity; import org.apache.nifi.web.api.entity.ControllerServiceEntity; import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentsEntity; import org.apache.nifi.web.api.entity.ControllerServicesEntity; @@ -305,6 +313,7 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C public static final Pattern PROCESSORS_URI_PATTERN = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/processors"); public static final Pattern PROCESSOR_URI_PATTERN = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/processors/[a-f0-9\\-]{36}"); + public static final Pattern PROCESSOR_STATE_URI_PATTERN = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/processors/[a-f0-9\\-]{36}/state"); public static final Pattern CLUSTER_PROCESSOR_URI_PATTERN = Pattern.compile("/nifi-api/cluster/processors/[a-f0-9\\-]{36}"); public static final Pattern REMOTE_PROCESS_GROUPS_URI_PATTERN = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/remote-process-groups"); @@ -321,9 +330,11 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C public static final Pattern COUNTERS_URI = Pattern.compile("/nifi-api/controller/counters/[a-f0-9\\-]{36}"); public static final String CONTROLLER_SERVICES_URI = "/nifi-api/controller/controller-services/node"; public static final Pattern CONTROLLER_SERVICE_URI_PATTERN = Pattern.compile("/nifi-api/controller/controller-services/node/[a-f0-9\\-]{36}"); + public static final Pattern CONTROLLER_SERVICE_STATE_URI_PATTERN = Pattern.compile("/nifi-api/controller/controller-services/node/[a-f0-9\\-]{36}/state"); public static final Pattern CONTROLLER_SERVICE_REFERENCES_URI_PATTERN = Pattern.compile("/nifi-api/controller/controller-services/node/[a-f0-9\\-]{36}/references"); public static final String REPORTING_TASKS_URI = "/nifi-api/controller/reporting-tasks/node"; public static final Pattern REPORTING_TASK_URI_PATTERN = Pattern.compile("/nifi-api/controller/reporting-tasks/node/[a-f0-9\\-]{36}"); + public static final Pattern REPORTING_TASK_STATE_URI_PATTERN = Pattern.compile("/nifi-api/controller/reporting-tasks/node/[a-f0-9\\-]{36}/state"); @Deprecated public static final Pattern QUEUE_CONTENTS_URI = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/connections/[a-f0-9\\-]{36}/contents"); @@ -368,8 +379,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) { @@ -408,40 +421,6 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C } componentStatusSnapshotMillis = snapshotMillis; - Executors.newSingleThreadScheduledExecutor().scheduleWithFixedDelay(new Runnable() { - @Override - public void run() { - readLock.lock(); - try { - for (final Node node : nodes) { - if (Status.CONNECTED.equals(node.getStatus())) { - ComponentStatusRepository statusRepository = componentMetricsRepositoryMap.get(node.getNodeId()); - if (statusRepository == null) { - statusRepository = createComponentStatusRepository(); - componentMetricsRepositoryMap.put(node.getNodeId(), statusRepository); - } - - // ensure this node has a payload - if (node.getHeartbeat() != null && node.getHeartbeatPayload() != null) { - // if nothing has been captured or the current heartbeat is newer, capture it - comparing the heatbeat created timestamp - // is safe since its marked as XmlTransient so we're assured that its based off the same clock that created the last capture date - if (statusRepository.getLastCaptureDate() == null || node.getHeartbeat().getCreatedTimestamp() > statusRepository.getLastCaptureDate().getTime()) { - statusRepository.capture(node.getHeartbeatPayload().getProcessGroupStatus()); - } - } - } - } - } catch (final Throwable t) { - logger.warn("Unable to capture component metrics from Node heartbeats: " + t); - if (logger.isDebugEnabled()) { - logger.warn("", t); - } - } finally { - readLock.unlock("capture component metrics from node heartbeats"); - } - } - }, componentStatusSnapshotMillis, componentStatusSnapshotMillis, TimeUnit.MILLISECONDS); - remoteInputPort = properties.getRemoteInputPort(); if (remoteInputPort == null) { remoteSiteListener = null; @@ -465,11 +444,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. @@ -477,14 +462,14 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C processScheduler.setSchedulingAgent(SchedulingStrategy.CRON_DRIVEN, new QuartzSchedulingAgent(null, reportingTaskEngine, null, encryptor)); processScheduler.setMaxThreadCount(SchedulingStrategy.TIMER_DRIVEN, 10); processScheduler.setMaxThreadCount(SchedulingStrategy.CRON_DRIVEN, 10); + processScheduler.scheduleFrameworkTask(new CaptureComponentMetrics(), "Capture Component Metrics", componentStatusSnapshotMillis, componentStatusSnapshotMillis, TimeUnit.MILLISECONDS); - 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."); } @@ -539,6 +524,8 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C if (serializedReportingTasks != null && serializedReportingTasks.length > 0) { loadReportingTasks(serializedReportingTasks); } + + notifyComponentsConfigurationRestored(); } catch (final IOException ioe) { logger.warn("Failed to initialize cluster services due to: " + ioe, ioe); stop(); @@ -675,6 +662,25 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C } } + + private void notifyComponentsConfigurationRestored() { + for (final ControllerServiceNode serviceNode : getAllControllerServices()) { + final ControllerService service = serviceNode.getControllerServiceImplementation(); + + try (final NarCloseable nc = NarCloseable.withNarLoader()) { + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, service); + } + } + + for (final ReportingTaskNode taskNode : getAllReportingTasks()) { + final ReportingTask task = taskNode.getReportingTask(); + + try (final NarCloseable nc = NarCloseable.withNarLoader()) { + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, task); + } + } + } + /** * Services connection requests. If the data flow management service is unable to provide a current copy of the data flow, then the returned connection response will indicate the node should try * later. Otherwise, the connection response will contain the the flow and the node identifier. @@ -709,7 +715,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 @@ -1093,13 +1106,15 @@ 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); if (firstTimeAdded) { try (final NarCloseable x = NarCloseable.withNarLoader()) { ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, task); + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, taskNode.getReportingTask()); } catch (final Exception e) { throw new ComponentLifeCycleException("Failed to invoke On-Added Lifecycle methods of " + task, e); } @@ -1351,8 +1366,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) { @@ -1405,9 +1421,21 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C logRepository.addObserver(StandardProcessorNode.BULLETIN_OBSERVER_ID, LogLevel.WARN, new ControllerServiceLogObserver(getBulletinRepository(), serviceNode)); + if (firstTimeAdded) { + final ControllerService service = serviceNode.getControllerServiceImplementation(); + + try (final NarCloseable nc = NarCloseable.withNarLoader()) { + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, service); + } + } + return serviceNode; } + public StateManagerProvider getStateManagerProvider() { + return stateManagerProvider; + } + @Override public ControllerService getControllerService(String serviceIdentifier) { return controllerServiceProvider.getControllerService(serviceIdentifier); @@ -1845,6 +1873,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) { @@ -2404,6 +2433,10 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C return false; } + private static boolean isProcessorStateEndpoint(final URI uri, final String method) { + return "GET".equalsIgnoreCase(method) && PROCESSOR_STATE_URI_PATTERN.matcher(uri.getPath()).matches(); + } + private static boolean isProcessGroupEndpoint(final URI uri, final String method) { return ("GET".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method)) && PROCESS_GROUP_URI_PATTERN.matcher(uri.getPath()).matches(); } @@ -2471,6 +2504,10 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C return false; } + private static boolean isControllerServiceStateEndpoint(final URI uri, final String method) { + return "GET".equalsIgnoreCase(method) && CONTROLLER_SERVICE_STATE_URI_PATTERN.matcher(uri.getPath()).matches(); + } + private static boolean isControllerServiceReferenceEndpoint(final URI uri, final String method) { if (("GET".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method)) && CONTROLLER_SERVICE_REFERENCES_URI_PATTERN.matcher(uri.getPath()).matches()) { return true; @@ -2493,6 +2530,10 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C return false; } + private static boolean isReportingTaskStateEndpoint(final URI uri, final String method) { + return "GET".equalsIgnoreCase(method) && REPORTING_TASK_STATE_URI_PATTERN.matcher(uri.getPath()).matches(); + } + private static boolean isDropRequestEndpoint(final URI uri, final String method) { if ("DELETE".equalsIgnoreCase(method) && QUEUE_CONTENTS_URI.matcher(uri.getPath()).matches()) { return true; @@ -2506,13 +2547,14 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C } static boolean isResponseInterpreted(final URI uri, final String method) { - return isProcessorsEndpoint(uri, method) || isProcessorEndpoint(uri, method) + return isProcessorsEndpoint(uri, method) || isProcessorEndpoint(uri, method) || isProcessorStateEndpoint(uri, method) || isRemoteProcessGroupsEndpoint(uri, method) || isRemoteProcessGroupEndpoint(uri, method) || isProcessGroupEndpoint(uri, method) || isTemplateEndpoint(uri, method) || isFlowSnippetEndpoint(uri, method) || isProvenanceQueryEndpoint(uri, method) || isProvenanceEventEndpoint(uri, method) - || isControllerServicesEndpoint(uri, method) || isControllerServiceEndpoint(uri, method) || isControllerServiceReferenceEndpoint(uri, method) - || isReportingTasksEndpoint(uri, method) || isReportingTaskEndpoint(uri, method) + || isControllerServicesEndpoint(uri, method) || isControllerServiceEndpoint(uri, method) + || isControllerServiceReferenceEndpoint(uri, method) || isControllerServiceStateEndpoint(uri, method) + || isReportingTasksEndpoint(uri, method) || isReportingTaskEndpoint(uri, method) || isReportingTaskStateEndpoint(uri, method) || isDropRequestEndpoint(uri, method) || isListFlowFilesEndpoint(uri, method); } @@ -2531,6 +2573,28 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C processor.setValidationErrors(normalizedMergedValidationErrors(validationErrorMap, processorMap.size())); } + private void mergeComponentState(final ComponentStateDTO componentState, Map componentStateMap) { + final List localStateEntries = new ArrayList<>(); + + for (final Map.Entry nodeEntry : componentStateMap.entrySet()) { + final ComponentStateDTO nodeComponentState = nodeEntry.getValue(); + final NodeIdentifier nodeId = nodeEntry.getKey(); + final String nodeAddress = nodeId.getApiAddress() + ":" + nodeId.getApiPort(); + + final StateMapDTO nodeLocalStateMap = nodeComponentState.getLocalState(); + if (nodeLocalStateMap.getState() != null) { + for (final StateEntryDTO nodeStateEntry : nodeLocalStateMap.getState()) { + nodeStateEntry.setClusterNodeId(nodeId.getId()); + nodeStateEntry.setClusterNodeAddress(nodeAddress); + localStateEntries.add(nodeStateEntry); + } + } + } + + // add all the local state entries + componentState.getLocalState().setState(localStateEntries); + } + private void mergeProvenanceQueryResults(final ProvenanceDTO provenanceDto, final Map resultMap, final Set problematicResponses) { final ProvenanceResultsDTO results = provenanceDto.getResults(); final ProvenanceRequestDTO request = provenanceDto.getRequest(); @@ -3448,6 +3512,24 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C } mergeListingRequests(listingRequest, resultsMap); + clientResponse = new NodeResponse(clientResponse, responseEntity); + } else if (hasSuccessfulClientResponse && (isProcessorStateEndpoint(uri, method) || isControllerServiceStateEndpoint(uri, method) || isReportingTaskStateEndpoint(uri, method))) { + final ComponentStateEntity responseEntity = clientResponse.getClientResponse().getEntity(ComponentStateEntity.class); + final ComponentStateDTO componentState = responseEntity.getComponentState(); + + final Map resultsMap = new HashMap<>(); + for (final NodeResponse nodeResponse : updatedNodesMap.values()) { + if (problematicNodeResponses.contains(nodeResponse)) { + continue; + } + + final ComponentStateEntity nodeResponseEntity = nodeResponse == clientResponse ? responseEntity : nodeResponse.getClientResponse().getEntity(ComponentStateEntity.class); + final ComponentStateDTO nodeComponentState = nodeResponseEntity.getComponentState(); + + resultsMap.put(nodeResponse.getNodeId(), nodeComponentState); + } + mergeComponentState(componentState, resultsMap); + clientResponse = new NodeResponse(clientResponse, responseEntity); } else { if (!nodeResponsesToDrain.isEmpty()) { @@ -3672,7 +3754,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) { @@ -3688,32 +3770,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 { @@ -3907,13 +3989,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); } @@ -4487,4 +4569,41 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C public Set getControllerServiceIdentifiers(final Class serviceType) { return controllerServiceProvider.getControllerServiceIdentifiers(serviceType); } + + /** + * Captures snapshots of components' metrics + */ + private class CaptureComponentMetrics implements Runnable { + @Override + public void run() { + readLock.lock(); + try { + for (final Node node : nodes) { + if (Status.CONNECTED.equals(node.getStatus())) { + ComponentStatusRepository statusRepository = componentMetricsRepositoryMap.get(node.getNodeId()); + if (statusRepository == null) { + statusRepository = createComponentStatusRepository(); + componentMetricsRepositoryMap.put(node.getNodeId(), statusRepository); + } + + // ensure this node has a payload + if (node.getHeartbeat() != null && node.getHeartbeatPayload() != null) { + // if nothing has been captured or the current heartbeat is newer, capture it - comparing the heatbeat created timestamp + // is safe since its marked as XmlTransient so we're assured that its based off the same clock that created the last capture date + if (statusRepository.getLastCaptureDate() == null || node.getHeartbeat().getCreatedTimestamp() > statusRepository.getLastCaptureDate().getTime()) { + statusRepository.capture(node.getHeartbeatPayload().getProcessGroupStatus()); + } + } + } + } + } catch (final Throwable t) { + logger.warn("Unable to capture component metrics from Node heartbeats: " + t); + if (logger.isDebugEnabled()) { + logger.warn("", t); + } + } finally { + readLock.unlock("capture component metrics from node heartbeats"); + } + } + } } 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/connectable/Connectable.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/Connectable.java index 60b225a56d..1a26f39923 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/Connectable.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/Connectable.java @@ -270,5 +270,7 @@ public interface Connectable extends Triggerable { void verifyCanDisable() throws IllegalStateException; + void verifyCanClearState() throws IllegalStateException; + SchedulingStrategy getSchedulingStrategy(); } 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 df93cc0838..924e61e8ce 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 @@ -117,7 +117,7 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone try { component.onPropertyModified(descriptor, oldValue, value); - } catch (final Throwable t) { + } catch (final Exception e) { // nothing really to do here... } } @@ -161,7 +161,12 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone } } - component.onPropertyModified(descriptor, value, null); + try { + component.onPropertyModified(descriptor, value, null); + } catch (final Exception e) { + // nothing really to do here... + } + return true; } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java index 50ba12a6df..9f86f08c8b 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java @@ -605,4 +605,8 @@ public abstract class AbstractPort implements Port { throw new IllegalStateException(this + " has " + threadCount + " threads still active"); } } + + @Override + public void verifyCanClearState() { + } } 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..0fcccddc4c 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,6 +34,12 @@ public interface ConfiguredComponent { public void setAnnotationData(String data); + /** + * 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 + */ public void setProperty(String name, String value); /** diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ReportingTaskNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ReportingTaskNode.java index c2adf01057..86a48e72eb 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ReportingTaskNode.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ReportingTaskNode.java @@ -98,4 +98,6 @@ public interface ReportingTaskNode extends ConfiguredComponent { void verifyCanDelete(); void verifyCanUpdate(); + + void verifyCanClearState(); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java index 0f596d88ae..647e4d5605 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java @@ -531,6 +531,10 @@ public class StandardFunnel implements Funnel { public void verifyCanDisable() { } + @Override + public void verifyCanClearState() { + } + @Override public SchedulingStrategy getSchedulingStrategy() { return SchedulingStrategy.TIMER_DRIVEN; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java index 4f1a961d37..e91ba9ae4e 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java @@ -134,6 +134,8 @@ public interface ControllerServiceNode extends ConfiguredComponent { void verifyCanUpdate(); + void verifyCanClearState(); + /** * Returns 'true' if this service is active. The service is considered to be * active if and only if it's diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/logging/LogRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/logging/LogRepository.java index 92091da1f6..54bb6720d4 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/logging/LogRepository.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/logging/LogRepository.java @@ -64,4 +64,16 @@ public interface LogRepository { * Removes all LogObservers from this Repository */ void removeAllObservers(); + + /** + * Sets the current logger for the component + * + * @param logger the logger to use + */ + void setLogger(ComponentLog logger); + + /** + * @return the current logger for the component + */ + ComponentLog getLogger(); } 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 dc694ebb72..b6e8e82b8b 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 @@ -119,6 +119,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 840df947cd..dab2d3dd52 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 @@ -22,6 +22,7 @@ import org.apache.nifi.action.Action; import org.apache.nifi.admin.service.AuditService; import org.apache.nifi.admin.service.UserService; import org.apache.nifi.annotation.lifecycle.OnAdded; +import org.apache.nifi.annotation.lifecycle.OnConfigurationRestored; import org.apache.nifi.annotation.lifecycle.OnRemoved; import org.apache.nifi.annotation.lifecycle.OnShutdown; import org.apache.nifi.annotation.notification.OnPrimaryNodeStateChange; @@ -37,6 +38,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 +90,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,6 +177,7 @@ 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; @@ -205,6 +210,7 @@ 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.LockSupport; import java.util.concurrent.locks.ReentrantReadWriteLock; import static java.util.Objects.requireNonNull; @@ -251,9 +257,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 +428,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 +484,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 +511,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 @@ -582,6 +608,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R externalSiteListener.start(); } + notifyComponentsConfigurationRestored(); + timerDrivenEngineRef.get().scheduleWithFixedDelay(new Runnable() { @Override public void run() { @@ -602,6 +630,31 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R } } + private void notifyComponentsConfigurationRestored() { + for (final ProcessorNode procNode : getGroup(getRootGroupId()).findAllProcessors()) { + final Processor processor = procNode.getProcessor(); + try (final NarCloseable nc = NarCloseable.withNarLoader()) { + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, processor); + } + } + + for (final ControllerServiceNode serviceNode : getAllControllerServices()) { + final ControllerService service = serviceNode.getControllerServiceImplementation(); + + try (final NarCloseable nc = NarCloseable.withNarLoader()) { + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, service); + } + } + + for (final ReportingTaskNode taskNode : getAllReportingTasks()) { + final ReportingTask task = taskNode.getReportingTask(); + + try (final NarCloseable nc = NarCloseable.withNarLoader()) { + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, task); + } + } + } + /** *

* Causes any processors that were added to the flow with a 'delayStart' flag of true to now start @@ -836,7 +889,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); } /** @@ -883,6 +936,12 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R logRepository.removeObserver(StandardProcessorNode.BULLETIN_OBSERVER_ID); throw new ComponentLifeCycleException("Failed to invoke @OnAdded methods of " + procNode.getProcessor(), e); } + + if (firstTimeAdded) { + try (final NarCloseable nc = NarCloseable.withNarLoader()) { + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, procNode.getProcessor()); + } + } } return procNode; @@ -909,6 +968,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R final ProcessorLog processorLogger = new SimpleProcessLogger(identifier, processor); final ProcessorInitializationContext ctx = new StandardProcessorInitializationContext(identifier, processorLogger, this); processor.initialize(ctx); + + LogRepositoryFactory.getRepository(identifier).setLogger(processorLogger); return processor; } catch (final Throwable t) { throw new ProcessorInstantiationException(type, t); @@ -946,6 +1007,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 * @@ -1099,24 +1164,25 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R } else { this.timerDrivenEngineRef.get().shutdown(); this.eventDrivenEngineRef.get().shutdown(); - LOG.info("Initiated graceful shutdown of flow controller...waiting up to " + gracefulShutdownSeconds - + " seconds"); + LOG.info("Initiated graceful shutdown of flow controller...waiting up to " + gracefulShutdownSeconds + " seconds"); } clusterTaskExecutor.shutdown(); - // Trigger any processors' methods marked with @OnShutdown to be - // called + if (zooKeeperStateServer != null) { + zooKeeperStateServer.shutdown(); + } + + // Trigger any processors' methods marked with @OnShutdown to be called rootGroup.shutdown(); - // invoke any methods annotated with @OnShutdown on Controller - // Services + stateManagerProvider.shutdown(); + + // invoke any methods annotated with @OnShutdown on Controller Services for (final ControllerServiceNode serviceNode : getAllControllerServices()) { try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { - final ConfigurationContext configContext = new StandardConfigurationContext(serviceNode, - controllerServiceProvider, null); - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, - serviceNode.getControllerServiceImplementation(), configContext); + final ConfigurationContext configContext = new StandardConfigurationContext(serviceNode, controllerServiceProvider, null); + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, serviceNode.getControllerServiceImplementation(), configContext); } } @@ -1124,8 +1190,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R for (final ReportingTaskNode taskNode : getAllReportingTasks()) { final ConfigurationContext configContext = taskNode.getConfigurationContext(); try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, taskNode.getReportingTask(), - configContext); + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, taskNode.getReportingTask(), configContext); } } @@ -1139,14 +1204,14 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R try { flowFileRepository.close(); } catch (final Throwable t) { - LOG.warn("Unable to shut down FlowFileRepository due to {}", new Object[] { t }); + LOG.warn("Unable to shut down FlowFileRepository due to {}", new Object[] {t}); } if (this.timerDrivenEngineRef.get().isTerminated() && eventDrivenEngineRef.get().isTerminated()) { LOG.info("Controller has been terminated successfully."); } else { LOG.warn("Controller hasn't terminated properly. There exists an uninterruptable thread that " - + "will take an indeterminate amount of time to stop. Might need to kill the program manually."); + + "will take an indeterminate amount of time to stop. Might need to kill the program manually."); } if (externalSiteListener != null) { @@ -1174,7 +1239,6 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R } finally { readLock.unlock(); } - } /** @@ -2605,6 +2669,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R try (final NarCloseable x = NarCloseable.withNarLoader()) { ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, task); + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, taskNode.getReportingTask()); } catch (final Exception e) { throw new ComponentLifeCycleException("Failed to invoke On-Added Lifecycle methods of " + task, e); } @@ -2688,6 +2753,14 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R logRepository.addObserver(StandardProcessorNode.BULLETIN_OBSERVER_ID, LogLevel.WARN, new ControllerServiceLogObserver(getBulletinRepository(), serviceNode)); + if (firstTimeAdded) { + final ControllerService service = serviceNode.getControllerServiceImplementation(); + + try (final NarCloseable nc = NarCloseable.withNarLoader()) { + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, service); + } + } + return serviceNode; } @@ -3054,8 +3127,49 @@ 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); + final Bulletin bulletin = BulletinFactory.createBulletin("Embedded ZooKeeper Server", Severity.ERROR.name(), + "Unable to started embedded ZooKeeper Server. See logs for more details. Will continue trying to start embedded server."); + getBulletinRepository().addBulletin(bulletin); + + // We failed to start the server. Wait a bit and try again. + try { + Thread.sleep(TimeUnit.SECONDS.toMillis(5)); + } catch (final InterruptedException ie) { + // If we are interrupted, stop trying. + Thread.currentThread().interrupt(); + return; + } + + processScheduler.submitFrameworkTask(this); + } + } + }); + + // Give the server just a bit to start up, so that we don't get connection + // failures on startup if we are using the embedded ZooKeeper server. We need to launch + // the ZooKeeper Server in the background because ZooKeeper blocks indefinitely when we start + // the server. Unfortunately, we have no way to know when it's up & ready. So we wait 1 second. + // We could still get connection failures if we are on a slow machine but this at least makes it far + // less likely. If we do get connection failures, we will still reconnect, but we will get bulletins + // showing failures. This 1-second sleep is an attempt to at least make that occurrence rare. + LockSupport.parkNanos(TimeUnit.SECONDS.toNanos(1L)); + } } else { bulletinRepository.restoreDefaultBulletinProcessing(); + if (zooKeeperStateServer != null) { + zooKeeperStateServer.shutdown(); + } + stateManagerProvider.disableClusterProvider(); } final List remoteGroups = getGroup(getRootGroupId()).findAllRemoteProcessGroups(); @@ -3698,8 +3812,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..f03b013c47 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) { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java index aade16b343..2db506cb84 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java @@ -1303,6 +1303,11 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable } } + @Override + public void verifyCanClearState() throws IllegalStateException { + verifyCanUpdate(); + } + private void verifyNoActiveThreads() throws IllegalStateException { final int threadCount = processScheduler.getActiveThreadCount(this); if (threadCount > 0) { 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..c3eb0a08ed 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 @@ -217,6 +217,11 @@ public abstract class AbstractReportingTaskNode extends AbstractConfiguredCompon } } + @Override + public void verifyCanClearState() { + verifyCanUpdate(); + } + @Override public void verifyCanStart(final Set ignoredReferences) { switch (getScheduledState()) { 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..ef27fb5708 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/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..3f24ff1616 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 @@ -218,6 +218,11 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i } } + @Override + public void verifyCanClearState() { + verifyCanUpdate(); + } + @Override public String getComments() { readLock.lock(); 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 e31cfeba6f..77dc87e4c7 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); @@ -491,6 +499,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..639f8a25bb --- /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,92 @@ +/* + * 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; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.logging.LogRepository; +import org.apache.nifi.logging.LogRepositoryFactory; +import org.apache.nifi.processor.SimpleProcessLogger; + +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; + } + + private ComponentLog getLogger(final String componentId) { + final LogRepository repo = LogRepositoryFactory.getRepository(componentId); + final ComponentLog logger = (repo == null) ? null : repo.getLogger(); + if (repo == null || logger == null) { + return new SimpleProcessLogger(componentId, this); + } + + return logger; + } + + @Override + public StateMap getState(final Scope scope) throws IOException { + final StateMap stateMap = getProvider(scope).getState(componentId); + getLogger(componentId).debug("Returning {} State: {}", new Object[] {scope, stateMap}); + return stateMap; + } + + + @Override + public boolean replace(final StateMap oldValue, final Map newValue, final Scope scope) throws IOException { + final boolean replaced = getProvider(scope).replace(oldValue, newValue, componentId); + getLogger(componentId).debug("{} State from old value {} to new value {} was {}", new Object[] {scope, oldValue, newValue, replaced}); + return replaced; + } + + @Override + public void setState(final Map state, final Scope scope) throws IOException { + getLogger(componentId).debug("Setting {} State to {}", new Object[] {scope, state}); + getProvider(scope).setState(state, componentId); + } + + @Override + public void clear(final Scope scope) throws IOException { + getLogger(componentId).debug("Clearing {} State", new Object[] {scope}); + 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..672fd6f5ef --- /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,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.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; + } + + @Override + public String toString() { + return "StandardStateMap[version=" + version + ", values=" + 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..2e7f5c9484 --- /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,114 @@ +/* + * 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()) { + final boolean hasKey = entry.getKey() != null; + final boolean hasValue = entry.getValue() != null; + out.writeBoolean(hasKey); + if (hasKey) { + out.writeUTF(entry.getKey()); + } + + out.writeBoolean(hasValue); + if (hasValue) { + 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 boolean hasKey = in.readBoolean(); + final String key = hasKey ? in.readUTF() : null; + final boolean hasValue = in.readBoolean(); + final String value = hasValue ? in.readUTF() : null; + 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..7565d9a6b1 --- /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,143 @@ +/* + * 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.components.state.Scope; +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 Scope 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, Scope.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, Scope.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 Scope 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..f8d29ee6c4 --- /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,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.controller.state.config; + +import org.apache.nifi.components.state.Scope; + +import java.util.HashMap; +import java.util.Map; + +public class StateProviderConfiguration { + private final String id; + private final Scope scope; + private final String className; + private final Map properties; + + public StateProviderConfiguration(final String id, final String className, final Scope 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 Scope 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/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..ef46f55dfc --- /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,301 @@ +/* + * 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.commons.lang3.ArrayUtils; +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.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, Scope.LOCAL, properties); + } + + + private static StateProvider createClusteredStateProvider(final NiFiProperties properties) throws IOException, ConfigParseException { + final File configFile = properties.getStateManagementConfigFile(); + return createStateProvider(configFile, Scope.CLUSTER, properties); + } + + + private static StateProvider createStateProvider(final File configFile, final Scope 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 == Scope.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); + } + + if (!ArrayUtils.contains(provider.getSupportedScopes(), scope)) { + throw new RuntimeException("Cannot use " + providerDescription + " ("+providerClassName+") as it only supports scope(s) " + ArrayUtils.toString(provider.getSupportedScopes()) + " but " + + "instance" + + " is configured to use scope " + scope); + } + + 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..fc691fb181 --- /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,265 @@ +/* + * 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.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.state.Scope; +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 (standalone) state, backed by a write-ahead log + */ +public class WriteAheadLocalStateProvider extends AbstractStateProvider { + private static final Logger logger = LoggerFactory.getLogger(WriteAheadLocalStateProvider.class); + + private final StateMapSerDe serde; + private final ConcurrentMap componentProviders = new ConcurrentHashMap<>(); + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(new NamedThreadFactory()); + + 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); + + executor.scheduleWithFixedDelay(new CheckpointTask(), 2, 2, TimeUnit.MINUTES); + } + + @Override + public List getSupportedPropertyDescriptors() { + final List properties = new ArrayList<>(); + properties.add(PATH); + return properties; + } + + @Override + public synchronized void shutdown() { + executor.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); + } + + @Override + public Scope[] getSupportedScopes() { + return new Scope[]{Scope.LOCAL}; + } + + 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.getVersion() == -1L) { + // state has never been set so return false + return false; + } + + 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); + } + } + + private class CheckpointTask implements Runnable { + @Override + public void run() { + try { + logger.debug("Checkpointing Write-Ahead Log used to store components' state"); + + writeAheadLog.checkpoint(); + } catch (final IOException e) { + logger.error("Failed to checkpoint Write-Ahead Log used to store components' state", e); + } + } + } + + private static class NamedThreadFactory implements ThreadFactory { + private final ThreadFactory defaultFactory = Executors.defaultThreadFactory(); + + @Override + public Thread newThread(final Runnable r) { + final Thread t = defaultFactory.newThread(r); + t.setName("Write-Ahead Local State Provider Maintenance"); + t.setDaemon(true); + return t; + } + } +} 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..acc0bc3f51 --- /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,485 @@ +/* + * 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.nio.charset.StandardCharsets; +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.TimeUnit; + +import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.Validator; +import org.apache.nifi.components.state.Scope; +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.KeeperException.NoNodeException; +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.client.ConnectStringParser; +import org.apache.zookeeper.data.ACL; +import org.apache.zookeeper.data.Stat; + +/** + * ZooKeeperStateProvider utilizes a ZooKeeper based store, whether provided internally via configuration and enabling of the {@link org.apache.nifi.controller.state.server.ZooKeeperStateServer} + * or through an externally configured location. This implementation caters to a clustered NiFi environment and accordingly only provides {@link Scope#CLUSTER} scoping to enforce + * consistency across configuration interactions. + */ +public class ZooKeeperStateProvider extends AbstractStateProvider { + static final AllowableValue OPEN_TO_WORLD = new AllowableValue("Open", "Open", "ZNodes will be open to any ZooKeeper client."); + static final AllowableValue CREATOR_ONLY = new AllowableValue("CreatorOnly", "CreatorOnly", + "ZNodes will be accessible only by the creator. The creator will have full access to create, read, write, delete, and administer the ZNodes."); + + 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 hostname/IP and port tuples, such as \"host1:2181,host2:2181,127.0.0.1:2181\". If a port is not " + + "specified it defaults to the ZooKeeper client port default of 2181") + .addValidator(new Validator() { + @Override + public ValidationResult validate(String subject, String input, ValidationContext context) { + final String connectionString = context.getProperty(CONNECTION_STRING).getValue(); + try { + new ConnectStringParser(connectionString); + } catch (Exception e) { + return new ValidationResult.Builder().subject(subject).input(input).explanation("Invalid Connect String: " + connectionString).valid(false).build(); + } + return new ValidationResult.Builder().subject(subject).input(input).explanation("Valid Connect String").valid(true).build(); + } + }) + .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(); + static final PropertyDescriptor ACCESS_CONTROL = new PropertyDescriptor.Builder() + .name("Access Control") + .description("Specifies the Access Controls that will be placed on ZooKeeper ZNodes that are created by this State Provider") + .allowableValues(OPEN_TO_WORLD, CREATOR_ONLY) + .defaultValue(OPEN_TO_WORLD.getValue()) + .required(true) + .build(); + static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder() + .name("Username") + .description("A Username that can be used to set Access Controls on ZooKeeper ZNodes. In order to apply any Access Controls to ZNodes, either a username and password must be set, " + + "or NiFi must be configured to communicate with ZooKeeper via Kerberos.") + .addValidator(Validator.VALID) + .required(false) + .build(); + static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder() + .name("Password") + .description("A password that can be used in conjunction with the Username property to set Access Controls on ZooKeeper ZNodes. " + + "In order to apply any Access Controls to ZNodes, either a username and password must be set, " + + "or NiFi must be configured to communicate with ZooKeeper via Kerberos.") + .addValidator(Validator.VALID) + .required(false) + .sensitive(true) + .build(); + + private static final byte ENCODING_VERSION = 1; + + private ZooKeeper zooKeeper; + + // effectively final + private int timeoutMillis; + private String rootNode; + private String connectionString; + private byte[] auth; + private List acl; + + + public ZooKeeperStateProvider() { + } + + + @Override + public List getSupportedPropertyDescriptors() { + final List properties = new ArrayList<>(); + properties.add(CONNECTION_STRING); + properties.add(SESSION_TIMEOUT); + properties.add(ROOT_NODE); + properties.add(ACCESS_CONTROL); + properties.add(USERNAME); + properties.add(PASSWORD); + return properties; + } + + + @Override + protected Collection customValidate(final ValidationContext validationContext) { + final List validationFailures = new ArrayList<>(); + + final String username = validationContext.getProperty(USERNAME).getValue(); + if (username != null && !username.trim().isEmpty()) { + final String password = validationContext.getProperty(PASSWORD).getValue(); + if (password == null || password.trim().isEmpty()) { + validationFailures.add(new ValidationResult.Builder() + .input("") + .subject("Username and Password") + .valid(false) + .explanation("If the Username is set, the Password must also be set") + .build()); + } + } + return validationFailures; + } + + @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(); + + final String username = context.getProperty(USERNAME).getValue(); + if (username == null) { + auth = null; + } else { + final String password = context.getProperty(PASSWORD).getValue(); + auth = (username + ":" + password).getBytes(StandardCharsets.UTF_8); + } + + if (context.getProperty(ACCESS_CONTROL).getValue().equalsIgnoreCase(CREATOR_ONLY.getValue())) { + acl = Ids.CREATOR_ALL_ACL; + } else { + acl = Ids.OPEN_ACL_UNSAFE; + } + } + + @Override + public synchronized void shutdown() { + if (zooKeeper != null) { + try { + zooKeeper.close(); + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + + zooKeeper = null; + } + + // visible for testing + 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) { + } + }); + + if (auth != null) { + zooKeeper.addAuthInfo("digest", auth); + } + } + + 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 because 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 + final Code exceptionCode = ke.code(); + if (Code.NONODE == exceptionCode) { + return; + } + if (Code.SESSIONEXPIRED == exceptionCode) { + invalidateClient(); + onComponentRemoved(componentId); + return; + } + + throw new IOException("Unable to remove state for component with ID '" + componentId + " with exception code " + exceptionCode, 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 Scope[] getSupportedScopes() { + return new Scope[]{Scope.CLUSTER}; + } + + @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.writeByte(ENCODING_VERSION); + dos.writeInt(stateValues.size()); + for (final Map.Entry entry : stateValues.entrySet()) { + final boolean hasKey = entry.getKey() != null; + final boolean hasValue = entry.getValue() != null; + dos.writeBoolean(hasKey); + if (hasKey) { + dos.writeUTF(entry.getKey()); + } + + dos.writeBoolean(hasValue); + if (hasValue) { + 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 byte encodingVersion = dis.readByte(); + 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 boolean hasKey = dis.readBoolean(); + final String key = hasKey ? dis.readUTF() : null; + + final boolean hasValue = dis.readBoolean(); + final String value = hasValue ? dis.readUTF() : null; + stateValues.put(key, value); + } + + return new StandardStateMap(stateValues, recordVersion); + } + } + + private void setState(final Map stateValues, final int version, final String componentId) throws IOException { + try { + setState(stateValues, version, componentId, true); + } catch (final NoNodeException nne) { + // should never happen because we are passing 'true' for allowNodeCreation + throw new IOException("Unable to create Node in ZooKeeper to set state for component with ID " + componentId, nne); + } + } + + /** + * Sets the component state to the given stateValues if and only if the version is equal to the version currently + * tracked by ZooKeeper (or if the version is -1, in which case the state will be updated regardless of the version). + * + * @param stateValues the new values to set + * @param version the expected version of the ZNode + * @param componentId the ID of the component whose state is being updated + * @param allowNodeCreation if true and the corresponding ZNode does not exist in ZooKeeper, it will be created; if false + * and the corresponding node does not exist in ZooKeeper, a {@link KeeperException.NoNodeException} will be thrown + * + * @throws IOException if unable to communicate with ZooKeeper + * @throws NoNodeException if the corresponding ZNode does not exist in ZooKeeper and allowNodeCreation is set to false + */ + private void setState(final Map stateValues, final int version, final String componentId, final boolean allowNodeCreation) throws IOException, NoNodeException { + verifyEnabled(); + + try { + final String path = getComponentPath(componentId); + final byte[] data = serialize(stateValues); + + final ZooKeeper keeper = getZooKeeper(); + try { + keeper.setData(path, data, version); + } catch (final NoNodeException nne) { + if (allowNodeCreation) { + createNode(path, data); + return; + } else { + throw nne; + } + } + } 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 NoNodeException nne) { + throw nne; + } catch (final KeeperException ke) { + if (Code.SESSIONEXPIRED == ke.code()) { + invalidateClient(); + setState(stateValues, version, componentId, allowNodeCreation); + return; + } + if (Code.NODEEXISTS == ke.code()) { + setState(stateValues, version, componentId, allowNodeCreation); + return; + } + + 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) { + final Code exceptionCode = ke.code(); + if (Code.NONODE == exceptionCode) { + final String parentPath = StringUtils.substringBeforeLast(path, "/"); + createNode(parentPath, null); + createNode(path, data); + return; + } + if (Code.SESSIONEXPIRED == exceptionCode) { + invalidateClient(); + createNode(path, data); + return; + } + + // Node already exists. Node must have been created by "someone else". Just set the data. + if (Code.NODEEXISTS == exceptionCode) { + try { + getZooKeeper().setData(path, data, -1); + return; + } 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); + return; + } + } 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) { + final Code exceptionCode = ke.code(); + if (Code.NONODE == exceptionCode) { + return new StandardStateMap(null, -1L); + } + if (Code.SESSIONEXPIRED == exceptionCode) { + invalidateClient(); + return getState(componentId); + } + + throw new IOException("Failed to obtain value from ZooKeeper for component with ID " + componentId + " with exception code " + exceptionCode, 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, false); + return true; + } catch (final NoNodeException nne) { + return false; + } catch (final IOException ioe) { + final Throwable cause = ioe.getCause(); + if (cause != null && cause instanceof KeeperException) { + final KeeperException ke = (KeeperException) cause; + if (Code.BADVERSION == ke.code()) { + return false; + } + } + + throw ioe; + } + } + + + @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..0fc9ec2ce4 --- /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,176 @@ +/* + * 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.ServerCnxnFactory; +import org.apache.zookeeper.server.ServerConfig; +import org.apache.zookeeper.server.ZKDatabase; +import org.apache.zookeeper.server.ZooKeeperServer; +import org.apache.zookeeper.server.ZooKeeperServerMain; +import org.apache.zookeeper.server.persistence.FileTxnSnapLog; +import org.apache.zookeeper.server.quorum.QuorumPeer; +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 final QuorumPeerConfig quorumPeerConfig; + private volatile boolean started = false; + + private ServerCnxnFactory connectionFactory; + private FileTxnSnapLog transactionLog; + private ZooKeeperServer embeddedZkServer; + private QuorumPeer quorumPeer; + + private ZooKeeperStateServer(final Properties zkProperties) throws IOException, ConfigException { + quorumPeerConfig = new QuorumPeerConfig(); + quorumPeerConfig.parseProperties(zkProperties); + } + + public synchronized void start() throws IOException { + if (quorumPeerConfig.isDistributed()) { + startDistributed(); + } else { + startStandalone(); + } + } + + private void startStandalone() throws IOException { + logger.info("Starting Embedded ZooKeeper Server"); + + final ServerConfig config = new ServerConfig(); + config.readFrom(quorumPeerConfig); + try { + started = true; + + transactionLog = new FileTxnSnapLog(new File(config.getDataLogDir()), new File(config.getDataDir())); + + embeddedZkServer = new ZooKeeperServer(); + embeddedZkServer.setTxnLogFactory(transactionLog); + embeddedZkServer.setTickTime(config.getTickTime()); + embeddedZkServer.setMinSessionTimeout(config.getMinSessionTimeout()); + embeddedZkServer.setMaxSessionTimeout(config.getMaxSessionTimeout()); + + connectionFactory = ServerCnxnFactory.createFactory(); + connectionFactory.configure(config.getClientPortAddress(), config.getMaxClientCnxns()); + connectionFactory.startup(embeddedZkServer); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + logger.warn("Embedded ZooKeeper Server interrupted", e); + } catch (final IOException ioe) { + throw new IOException("Failed to start embedded ZooKeeper Server", ioe); + } catch (final Exception e) { + throw new RuntimeException("Failed to start embedded ZooKeeper Server", e); + } + } + + private void startDistributed() throws IOException { + logger.info("Starting Embedded ZooKeeper Peer"); + + try { + started = true; + + transactionLog = new FileTxnSnapLog(new File(quorumPeerConfig.getDataLogDir()), new File(quorumPeerConfig.getDataDir())); + + connectionFactory = ServerCnxnFactory.createFactory(); + connectionFactory.configure(quorumPeerConfig.getClientPortAddress(), quorumPeerConfig.getMaxClientCnxns()); + + quorumPeer = new QuorumPeer(); + quorumPeer.setClientPortAddress(quorumPeerConfig.getClientPortAddress()); + quorumPeer.setTxnFactory(new FileTxnSnapLog(new File(quorumPeerConfig.getDataLogDir()), new File(quorumPeerConfig.getDataDir()))); + quorumPeer.setQuorumPeers(quorumPeerConfig.getServers()); + quorumPeer.setElectionType(quorumPeerConfig.getElectionAlg()); + quorumPeer.setMyid(quorumPeerConfig.getServerId()); + quorumPeer.setTickTime(quorumPeerConfig.getTickTime()); + quorumPeer.setMinSessionTimeout(quorumPeerConfig.getMinSessionTimeout()); + quorumPeer.setMaxSessionTimeout(quorumPeerConfig.getMaxSessionTimeout()); + quorumPeer.setInitLimit(quorumPeerConfig.getInitLimit()); + quorumPeer.setSyncLimit(quorumPeerConfig.getSyncLimit()); + quorumPeer.setQuorumVerifier(quorumPeerConfig.getQuorumVerifier()); + quorumPeer.setCnxnFactory(connectionFactory); + quorumPeer.setZKDatabase(new ZKDatabase(quorumPeer.getTxnFactory())); + quorumPeer.setLearnerType(quorumPeerConfig.getPeerType()); + quorumPeer.setSyncEnabled(quorumPeerConfig.getSyncEnabled()); + quorumPeer.setQuorumListenOnAllIPs(quorumPeerConfig.getQuorumListenOnAllIPs()); + + quorumPeer.start(); + } catch (final IOException ioe) { + throw new IOException("Failed to start embedded ZooKeeper Peer", ioe); + } catch (final Exception e) { + throw new RuntimeException("Failed to start embedded ZooKeeper Peer", e); + } + } + + @Override + public synchronized void shutdown() { + if (started) { + started = false; + + if (transactionLog != null) { + try { + transactionLog.close(); + } catch (final IOException ioe) { + logger.warn("Failed to close Transaction Log", ioe); + } + } + + if (connectionFactory != null) { + connectionFactory.shutdown(); + } + + if (quorumPeer != null && quorumPeer.isRunning()) { + quorumPeer.shutdown(); + } + + if (embeddedZkServer != null && embeddedZkServer.isRunning()) { + embeddedZkServer.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/logging/repository/StandardLogRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/repository/StandardLogRepository.java index f8d37e5794..f6c55b6d99 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/repository/StandardLogRepository.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/repository/StandardLogRepository.java @@ -23,6 +23,7 @@ import java.util.Map; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.logging.LogLevel; import org.apache.nifi.logging.LogMessage; import org.apache.nifi.logging.LogObserver; @@ -45,6 +46,8 @@ public class StandardLogRepository implements LogRepository { private final Logger logger = LoggerFactory.getLogger(StandardLogRepository.class); + private volatile ComponentLog componentLogger; + @Override public void addLogMessage(final LogLevel level, final String message) { addLogMessage(level, message, (Throwable) null); @@ -170,4 +173,14 @@ public class StandardLogRepository implements LogRepository { writeLock.unlock(); } } + + @Override + public void setLogger(final ComponentLog componentLogger) { + this.componentLogger = componentLogger; + } + + @Override + public ComponentLog getLogger() { + return componentLogger; + } } 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..04d175a934 --- /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.controller.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..f507eeab6a 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,7 +119,7 @@ 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); @@ -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..0dcacb537e 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,7 +97,7 @@ 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); @@ -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: // @@ -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: // @@ -280,7 +304,7 @@ 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); @@ -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/TestStateMapSerDe.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/state/TestStateMapSerDe.java new file mode 100644 index 0000000000..63bfbade47 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/state/TestStateMapSerDe.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.controller.state; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +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.junit.Test; +import org.wali.UpdateType; + +public class TestStateMapSerDe { + + @Test + public void testCreateRoundTrip() throws IOException { + final String componentId = "1234"; + + final StateMapSerDe serde = new StateMapSerDe(); + final Map stateValues = new HashMap<>(); + stateValues.put("abc", "xyz"); + stateValues.put("cba", "zyx"); + final StateMap stateMap = new StandardStateMap(stateValues, 3L); + final StateMapUpdate record = new StateMapUpdate(stateMap, componentId, UpdateType.CREATE); + + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + try (final DataOutputStream out = new DataOutputStream(baos)) { + serde.serializeRecord(record, out); + } + + final StateMapUpdate update; + final ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray()); + try (final DataInputStream in = new DataInputStream(bais)) { + update = serde.deserializeRecord(in, serde.getVersion()); + } + + assertNotNull(update); + assertEquals(componentId, update.getComponentId()); + assertEquals(UpdateType.CREATE, update.getUpdateType()); + final StateMap recoveredStateMap = update.getStateMap(); + + assertEquals(3L, recoveredStateMap.getVersion()); + assertEquals(stateValues, recoveredStateMap.toMap()); + } +} 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..1cd1f37792 --- /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,192 @@ +/* + * 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()); + } + + @Test + public void testReplaceWithNonExistingValue() throws Exception { + final StateProvider provider = getProvider(); + StateMap stateMap = provider.getState(componentId); + assertNotNull(stateMap); + + final Map newValue = new HashMap<>(); + newValue.put("value", "value"); + + final boolean replaced = provider.replace(stateMap, newValue, componentId); + assertFalse(replaced); + } + + @Test + public void testReplaceWithNonExistingValueAndVersionGreaterThanNegativeOne() throws Exception { + final StateProvider provider = getProvider(); + final StateMap stateMap = new StateMap() { + @Override + public long getVersion() { + return 4; + } + + @Override + public String get(String key) { + return null; + } + + @Override + public Map toMap() { + return Collections.emptyMap(); + } + }; + + final Map newValue = new HashMap<>(); + newValue.put("value", "value"); + + final boolean replaced = provider.replace(stateMap, newValue, componentId); + assertFalse(replaced); + } + + 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..7e03a9ce51 --- /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,205 @@ +/* + * 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 static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +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.apache.zookeeper.KeeperException; +import org.apache.zookeeper.KeeperException.Code; +import org.apache.zookeeper.ZooDefs.Perms; +import org.apache.zookeeper.data.ACL; +import org.apache.zookeeper.data.Stat; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.testng.Assert; + +public class TestZooKeeperStateProvider extends AbstractTestStateProvider { + + private StateProvider provider; + private TestingServer zkServer; + + private static final Map defaultProperties = new HashMap<>(); + + static { + defaultProperties.put(ZooKeeperStateProvider.SESSION_TIMEOUT, "3 secs"); + defaultProperties.put(ZooKeeperStateProvider.ROOT_NODE, "/nifi/team1/testing"); + defaultProperties.put(ZooKeeperStateProvider.ACCESS_CONTROL, ZooKeeperStateProvider.OPEN_TO_WORLD.getValue()); + } + + + @Before + public void setup() throws Exception { + zkServer = new TestingServer(true); + zkServer.start(); + + final Map properties = new HashMap<>(defaultProperties); + properties.put(ZooKeeperStateProvider.CONNECTION_STRING, zkServer.getConnectString()); + this.provider = createProvider(properties); + } + + private void initializeProvider(final ZooKeeperStateProvider provider, final Map properties) throws IOException { + provider.initialize(new StateProviderInitializationContext() { + @Override + public String getIdentifier() { + return "Unit Test Provider Initialization Context"; + } + + @Override + public Map getProperties() { + final Map propValueMap = new HashMap<>(); + for (final Map.Entry entry : properties.entrySet()) { + propValueMap.put(entry.getKey(), new StandardPropertyValue(entry.getValue(), null)); + } + return propValueMap; + } + + @Override + public PropertyValue getProperty(final PropertyDescriptor property) { + final String prop = properties.get(property); + return new StandardPropertyValue(prop, null); + } + + @Override + public SSLContext getSSLContext() { + return null; + } + }); + } + + private ZooKeeperStateProvider createProvider(final Map properties) throws Exception { + final ZooKeeperStateProvider provider = new ZooKeeperStateProvider(); + initializeProvider(provider, properties); + provider.enable(); + return provider; + } + + @After + public void clear() throws IOException { + try { + getProvider().onComponentRemoved(componentId); + getProvider().disable(); + getProvider().shutdown(); + } finally { + if (zkServer != null) { + zkServer.stop(); + zkServer.close(); + } + } + } + + + @Override + protected StateProvider getProvider() { + return provider; + } + + @Test + public void testWithUsernameAndPasswordCreatorOnly() throws Exception { + final Map properties = new HashMap<>(defaultProperties); + properties.put(ZooKeeperStateProvider.CONNECTION_STRING, zkServer.getConnectString()); + properties.put(ZooKeeperStateProvider.USERNAME, "nifi"); + properties.put(ZooKeeperStateProvider.PASSWORD, "nifi"); + properties.put(ZooKeeperStateProvider.ACCESS_CONTROL, ZooKeeperStateProvider.CREATOR_ONLY.getValue()); + + final ZooKeeperStateProvider authorizedProvider = createProvider(properties); + + try { + final Map state = new HashMap<>(); + state.put("testWithUsernameAndPasswordCreatorOnly", "my value"); + authorizedProvider.setState(state, componentId); + + final List acls = authorizedProvider.getZooKeeper().getACL(properties.get(ZooKeeperStateProvider.ROOT_NODE) + "/components/" + componentId, new Stat()); + assertNotNull(acls); + assertEquals(1, acls.size()); + final ACL acl = acls.get(0); + assertEquals(Perms.ALL, acl.getPerms()); + // ID is our username: + assertEquals("nifi:RuSeH3tpzgba3p9WrG/UpiSIsGg=", acl.getId().getId()); + + final Map stateValues = authorizedProvider.getState(componentId).toMap(); + assertEquals(state, stateValues); + + // ensure that our default provider cannot access the data, since it has not authenticated + try { + this.provider.getState(componentId); + Assert.fail("Expected an IOException but it wasn't thrown"); + } catch (final IOException ioe) { + final Throwable cause = ioe.getCause(); + assertTrue(cause instanceof KeeperException); + final KeeperException ke = (KeeperException) cause; + assertEquals(Code.NOAUTH, ke.code()); + } + } finally { + authorizedProvider.onComponentRemoved(componentId); + authorizedProvider.disable(); + authorizedProvider.shutdown(); + } + } + + @Test + public void testWithUsernameAndPasswordOpen() throws Exception { + final Map properties = new HashMap<>(defaultProperties); + properties.put(ZooKeeperStateProvider.CONNECTION_STRING, zkServer.getConnectString()); + properties.put(ZooKeeperStateProvider.USERNAME, "nifi"); + properties.put(ZooKeeperStateProvider.PASSWORD, "nifi"); + properties.put(ZooKeeperStateProvider.ACCESS_CONTROL, ZooKeeperStateProvider.OPEN_TO_WORLD.getValue()); + + final ZooKeeperStateProvider authorizedProvider = createProvider(properties); + + try { + final Map state = new HashMap<>(); + state.put("testWithUsernameAndPasswordOpen", "my value"); + authorizedProvider.setState(state, componentId); + + final List acls = authorizedProvider.getZooKeeper().getACL(properties.get(ZooKeeperStateProvider.ROOT_NODE) + "/components/" + componentId, new Stat()); + assertNotNull(acls); + assertEquals(1, acls.size()); + final ACL acl = acls.get(0); + assertEquals(Perms.ALL, acl.getPerms()); + assertEquals("anyone", acl.getId().getId()); + + final Map stateValues = authorizedProvider.getState(componentId).toMap(); + assertEquals(state, stateValues); + + // ensure that our default provider can also access the data, since it has not authenticated + final Map unauthStateValues = this.provider.getState(componentId).toMap(); + assertEquals(state, unauthStateValues); + } finally { + authorizedProvider.onComponentRemoved(componentId); + authorizedProvider.disable(); + authorizedProvider.shutdown(); + } + } +} 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..665b22b983 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/state-management.xml @@ -0,0 +1,68 @@ + + + + + + + local-provider + org.apache.nifi.controller.state.providers.local.WriteAheadLocalStateProvider + ./state/local + + + + + zk-provider + org.apache.nifi.controller.state.providers.zookeeper.ZooKeeperStateProvider + + /nifi + 30 seconds + CreatorOnly + nifi + nifi + + \ 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..fa1d8855fc --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/zookeeper.properties @@ -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. +# +# +# + +clientPort=2181 +initLimit=10 +autopurge.purgeInterval=24 +syncLimit=5 +tickTime=2000 +dataDir=./state/zookeeper +autopurge.snapRetainCount=30 + +# +# Specifies the servers that are part of this zookeeper ensemble. For +# every NiFi instance running an embedded zookeeper, there needs to be +# a server entry below. For instance: +# +# server.1=nifi-node1-hostname:2888:3888 +# server.2=nifi-node2-hostname:2888:3888 +# server.3=nifi-node3-hostname:2888:3888 +# +# The index of the server corresponds to the myid file that gets created +# in the dataDir of each node running an embedded zookeeper. See the +# administration guide for more details. +# + +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/audit/ComponentStateAuditor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ComponentStateAuditor.java new file mode 100644 index 0000000000..5a5d1601b3 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ComponentStateAuditor.java @@ -0,0 +1,185 @@ +/* + * 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.audit; + +import org.apache.nifi.action.Action; +import org.apache.nifi.action.Component; +import org.apache.nifi.action.FlowChangeAction; +import org.apache.nifi.action.Operation; +import org.apache.nifi.action.component.details.FlowChangeExtensionDetails; +import org.apache.nifi.components.state.StateMap; +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.ReportingTaskNode; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.user.NiFiUser; +import org.apache.nifi.web.security.user.NiFiUserUtils; +import org.aspectj.lang.ProceedingJoinPoint; +import org.aspectj.lang.annotation.Around; +import org.aspectj.lang.annotation.Aspect; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Date; + +@Aspect +public class ComponentStateAuditor extends NiFiAuditor { + + private static final Logger logger = LoggerFactory.getLogger(ComponentStateAuditor.class); + + /** + * Audits clearing of state from a Processor. + * + * @param proceedingJoinPoint join point + * @param processor the processor + * @throws java.lang.Throwable ex + */ + @Around("within(org.apache.nifi.web.dao.ComponentStateDAO+) && " + + "execution(void clearState(org.apache.nifi.controller.ProcessorNode)) && " + + "args(processor)") + public StateMap clearProcessorStateAdvice(ProceedingJoinPoint proceedingJoinPoint, ProcessorNode processor) throws Throwable { + + // update the processors state + final StateMap stateMap = (StateMap) proceedingJoinPoint.proceed(); + + // if no exception were thrown, add the clear action... + + // get the current user + NiFiUser user = NiFiUserUtils.getNiFiUser(); + + // ensure the user was found + if (user != null) { + Collection actions = new ArrayList<>(); + + // create the processor details + FlowChangeExtensionDetails processorDetails = new FlowChangeExtensionDetails(); + processorDetails.setType(processor.getProcessor().getClass().getSimpleName()); + + // create the clear action + FlowChangeAction configAction = new FlowChangeAction(); + configAction.setUserIdentity(user.getIdentity()); + configAction.setUserName(user.getUserName()); + configAction.setOperation(Operation.ClearState); + configAction.setTimestamp(new Date()); + configAction.setSourceId(processor.getIdentifier()); + configAction.setSourceName(processor.getName()); + configAction.setSourceType(Component.Processor); + configAction.setComponentDetails(processorDetails); + actions.add(configAction); + + // record the action + saveActions(actions, logger); + } + + return stateMap; + } + + /** + * Audits clearing of state from a Controller Service. + * + * @param proceedingJoinPoint join point + * @param controllerService the controller service + * @throws java.lang.Throwable ex + */ + @Around("within(org.apache.nifi.web.dao.ComponentStateDAO+) && " + + "execution(void clearState(org.apache.nifi.controller.service.ControllerServiceNode)) && " + + "args(controllerService)") + public StateMap clearControllerServiceStateAdvice(ProceedingJoinPoint proceedingJoinPoint, ControllerServiceNode controllerService) throws Throwable { + + // update the controller service state + final StateMap stateMap = (StateMap) proceedingJoinPoint.proceed(); + + // if no exception were thrown, add the clear action... + + // get the current user + NiFiUser user = NiFiUserUtils.getNiFiUser(); + + // ensure the user was found + if (user != null) { + Collection actions = new ArrayList<>(); + + // create the controller service details + FlowChangeExtensionDetails controllerServiceDetails = new FlowChangeExtensionDetails(); + controllerServiceDetails.setType(controllerService.getControllerServiceImplementation().getClass().getSimpleName()); + + // create the clear action + FlowChangeAction configAction = new FlowChangeAction(); + configAction.setUserIdentity(user.getIdentity()); + configAction.setUserName(user.getUserName()); + configAction.setOperation(Operation.ClearState); + configAction.setTimestamp(new Date()); + configAction.setSourceId(controllerService.getIdentifier()); + configAction.setSourceName(controllerService.getName()); + configAction.setSourceType(Component.ControllerService); + configAction.setComponentDetails(controllerServiceDetails); + actions.add(configAction); + + // record the action + saveActions(actions, logger); + } + + return stateMap; + } + + /** + * Audits clearing of state from a Processor. + * + * @param proceedingJoinPoint join point + * @param reportingTask the reporting task + * @throws java.lang.Throwable ex + */ + @Around("within(org.apache.nifi.web.dao.ComponentStateDAO+) && " + + "execution(void clearState(org.apache.nifi.controller.ReportingTaskNode)) && " + + "args(reportingTask)") + public StateMap clearReportingTaskStateAdvice(ProceedingJoinPoint proceedingJoinPoint, ReportingTaskNode reportingTask) throws Throwable { + + // update the reporting task state + final StateMap stateMap = (StateMap) proceedingJoinPoint.proceed(); + + // if no exception were thrown, add the clear action... + + // get the current user + NiFiUser user = NiFiUserUtils.getNiFiUser(); + + // ensure the user was found + if (user != null) { + Collection actions = new ArrayList<>(); + + // create the reporting task details + FlowChangeExtensionDetails reportingTaskDetails = new FlowChangeExtensionDetails(); + reportingTaskDetails.setType(reportingTask.getReportingTask().getClass().getSimpleName()); + + // create the clear action + FlowChangeAction configAction = new FlowChangeAction(); + configAction.setUserIdentity(user.getIdentity()); + configAction.setUserName(user.getUserName()); + configAction.setOperation(Operation.ClearState); + configAction.setTimestamp(new Date()); + configAction.setSourceId(reportingTask.getIdentifier()); + configAction.setSourceName(reportingTask.getName()); + configAction.setSourceType(Component.ReportingTask); + configAction.setComponentDetails(reportingTaskDetails); + actions.add(configAction); + + // record the action + saveActions(actions, logger); + } + + return stateMap; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java index 150dd67df8..bc5245c2ff 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java @@ -22,6 +22,7 @@ import org.apache.nifi.controller.service.ControllerServiceState; import org.apache.nifi.web.api.dto.BulletinBoardDTO; import org.apache.nifi.web.api.dto.BulletinQueryDTO; import org.apache.nifi.web.api.dto.ClusterDTO; +import org.apache.nifi.web.api.dto.ComponentStateDTO; import org.apache.nifi.web.api.dto.ComponentHistoryDTO; import org.apache.nifi.web.api.dto.ConnectionDTO; import org.apache.nifi.web.api.dto.ControllerConfigurationDTO; @@ -995,6 +996,85 @@ public interface NiFiServiceFacade { */ ConfigurationSnapshot deleteFunnel(Revision revision, String groupId, String funnelId); + // ---------------------------------------- + // Component state methods + // ---------------------------------------- + + /** + * Gets the state for the specified processor. + * + * @param groupId group + * @param processorId the processor id + * @return the component state + */ + ComponentStateDTO getProcessorState(String groupId, String processorId); + + /** + * Verifies the processor state could be cleared. + * + * @param groupId group + * @param processorId the processor id + */ + void verifyCanClearProcessorState(String groupId, String processorId); + + /** + * Clears the state for the specified processor. + * + * @param revision Revision to compare with current base revision + * @param groupId group + * @param processorId the processor id + * @return snapshot + */ + ConfigurationSnapshot clearProcessorState(Revision revision, String groupId, String processorId); + + /** + * Gets the state for the specified controller service. + * + * @param controllerServiceId the controller service id + * @return the component state + */ + ComponentStateDTO getControllerServiceState(String controllerServiceId); + + /** + * Verifies the controller service state could be cleared. + * + * @param controllerServiceId the controller service id + */ + void verifyCanClearControllerServiceState(String controllerServiceId); + + /** + * Clears the state for the specified controller service. + * + * @param revision Revision to compare with current base revision + * @param controllerServiceId the controller service id + * @return snapshot + */ + ConfigurationSnapshot clearControllerServiceState(Revision revision, String controllerServiceId); + + /** + * Gets the state for the specified reporting task. + * + * @param reportingTaskId the reporting task id + * @return the component state + */ + ComponentStateDTO getReportingTaskState(String reportingTaskId); + + /** + * Verifies the reporting task state could be cleared. + * + * @param reportingTaskId the reporting task id + */ + void verifyCanClearReportingTaskState(String reportingTaskId); + + /** + * Clears the state for the specified reporting task. + * + * @param revision Revision to compare with current base revision + * @param reportingTaskId the reporting task id + * @return snapshot + */ + ConfigurationSnapshot clearReportingTaskState(Revision revision, String reportingTaskId); + // ---------------------------------------- // Label methods // ---------------------------------------- diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacadeLock.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacadeLock.java index 70cf9957e0..44a55d873d 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacadeLock.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacadeLock.java @@ -43,6 +43,17 @@ public class NiFiServiceFacadeLock { } } + @Around("within(org.apache.nifi.web.NiFiServiceFacade+) && " + + "execution(* clear*(..))") + public Object clearLock(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { + writeLock.lock(); + try { + return proceedingJoinPoint.proceed(); + } finally { + writeLock.unlock(); + } + } + @Around("within(org.apache.nifi.web.NiFiServiceFacade+) && " + "execution(* delete*(..))") public Object deleteLock(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java index 142ff08898..945c67109e 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java @@ -39,6 +39,8 @@ import org.apache.nifi.cluster.protocol.NodeIdentifier; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationResult; import org.apache.nifi.components.Validator; +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateMap; import org.apache.nifi.connectable.Connection; import org.apache.nifi.connectable.Funnel; import org.apache.nifi.connectable.Port; @@ -81,6 +83,7 @@ import org.apache.nifi.web.api.dto.BulletinDTO; import org.apache.nifi.web.api.dto.BulletinQueryDTO; import org.apache.nifi.web.api.dto.ClusterDTO; import org.apache.nifi.web.api.dto.ComponentHistoryDTO; +import org.apache.nifi.web.api.dto.ComponentStateDTO; import org.apache.nifi.web.api.dto.ConnectionDTO; import org.apache.nifi.web.api.dto.ControllerConfigurationDTO; import org.apache.nifi.web.api.dto.ControllerDTO; @@ -792,6 +795,90 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { return dtoFactory.createCounterDto(controllerFacade.resetCounter(counterId)); } + @Override + public void verifyCanClearProcessorState(final String groupId, final String processorId) { + processorDAO.verifyClearState(groupId, processorId); + } + + @Override + public ConfigurationSnapshot clearProcessorState(final Revision revision, final String groupId, final String processorId) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public ConfigurationResult execute() { + // clear the state for the specified component + processorDAO.clearState(groupId, processorId); + + return new ConfigurationResult() { + @Override + public boolean isNew() { + return false; + } + + @Override + public ControllerConfigurationDTO getConfiguration() { + return null; + } + }; + } + }); + } + + @Override + public void verifyCanClearControllerServiceState(final String controllerServiceId) { + controllerServiceDAO.verifyClearState(controllerServiceId); + } + + @Override + public ConfigurationSnapshot clearControllerServiceState(final Revision revision, final String controllerServiceId) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public ConfigurationResult execute() { + // clear the state for the specified component + controllerServiceDAO.clearState(controllerServiceId); + + return new ConfigurationResult() { + @Override + public boolean isNew() { + return false; + } + + @Override + public ControllerConfigurationDTO getConfiguration() { + return null; + } + }; + } + }); + } + + @Override + public void verifyCanClearReportingTaskState(final String reportingTaskId) { + reportingTaskDAO.verifyClearState(reportingTaskId); + } + + @Override + public ConfigurationSnapshot clearReportingTaskState(final Revision revision, final String reportingTaskId) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public ConfigurationResult execute() { + // clear the state for the specified component + reportingTaskDAO.clearState(reportingTaskId); + + return new ConfigurationResult() { + @Override + public boolean isNew() { + return false; + } + + @Override + public ControllerConfigurationDTO getConfiguration() { + return null; + } + }; + } + }); + } + @Override public ConfigurationSnapshot deleteConnection(final Revision revision, final String groupId, final String connectionId) { return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { @@ -2099,6 +2186,36 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { return controllerStatus; } + @Override + public ComponentStateDTO getProcessorState(String groupId, String processorId) { + final StateMap clusterState = isClustered() ? processorDAO.getState(groupId, processorId, Scope.CLUSTER) : null; + final StateMap localState = processorDAO.getState(groupId, processorId, Scope.LOCAL); + + // processor will be non null as it was already found when getting the state + final ProcessorNode processor = processorDAO.getProcessor(groupId, processorId); + return dtoFactory.createComponentStateDTO(processorId, processor.getProcessor().getClass(), localState, clusterState); + } + + @Override + public ComponentStateDTO getControllerServiceState(String controllerServiceId) { + final StateMap clusterState = isClustered() ? controllerServiceDAO.getState(controllerServiceId, Scope.CLUSTER) : null; + final StateMap localState = controllerServiceDAO.getState(controllerServiceId, Scope.LOCAL); + + // controller service will be non null as it was already found when getting the state + final ControllerServiceNode controllerService = controllerServiceDAO.getControllerService(controllerServiceId); + return dtoFactory.createComponentStateDTO(controllerServiceId, controllerService.getControllerServiceImplementation().getClass(), localState, clusterState); + } + + @Override + public ComponentStateDTO getReportingTaskState(String reportingTaskId) { + final StateMap clusterState = isClustered() ? reportingTaskDAO.getState(reportingTaskId, Scope.CLUSTER) : null; + final StateMap localState = reportingTaskDAO.getState(reportingTaskId, Scope.LOCAL); + + // reporting task will be non null as it was already found when getting the state + final ReportingTaskNode reportingTask = reportingTaskDAO.getReportingTask(reportingTaskId); + return dtoFactory.createComponentStateDTO(reportingTaskId, reportingTask.getReportingTask().getClass(), localState, clusterState); + } + @Override public CountersDTO getCounters() { if (properties.isClusterManager()) { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java index 0f2cdce42c..685fac84c5 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java @@ -22,14 +22,34 @@ import com.wordnik.swagger.annotations.ApiParam; import com.wordnik.swagger.annotations.ApiResponse; import com.wordnik.swagger.annotations.ApiResponses; import com.wordnik.swagger.annotations.Authorization; -import java.net.URI; -import java.net.URISyntaxException; -import java.util.HashMap; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.UUID; +import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.cluster.manager.impl.WebClusterManager; +import org.apache.nifi.controller.ScheduledState; +import org.apache.nifi.controller.service.ControllerServiceState; +import org.apache.nifi.ui.extension.UiExtension; +import org.apache.nifi.ui.extension.UiExtensionMapping; +import org.apache.nifi.util.NiFiProperties; +import org.apache.nifi.web.ConfigurationSnapshot; +import org.apache.nifi.web.NiFiServiceFacade; +import org.apache.nifi.web.Revision; +import org.apache.nifi.web.UiExtensionType; +import org.apache.nifi.web.api.dto.ComponentStateDTO; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; +import org.apache.nifi.web.api.dto.ControllerServiceReferencingComponentDTO; +import org.apache.nifi.web.api.dto.PropertyDescriptorDTO; +import org.apache.nifi.web.api.dto.RevisionDTO; +import org.apache.nifi.web.api.entity.ComponentStateEntity; +import org.apache.nifi.web.api.entity.ControllerServiceEntity; +import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentsEntity; +import org.apache.nifi.web.api.entity.ControllerServicesEntity; +import org.apache.nifi.web.api.entity.PropertyDescriptorEntity; +import org.apache.nifi.web.api.request.ClientIdParameter; +import org.apache.nifi.web.api.request.LongParameter; +import org.apache.nifi.web.util.Availability; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.security.access.prepost.PreAuthorize; + import javax.servlet.ServletContext; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.Consumes; @@ -49,32 +69,14 @@ import javax.ws.rs.core.Context; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.MultivaluedMap; import javax.ws.rs.core.Response; -import org.apache.nifi.cluster.manager.impl.WebClusterManager; -import org.apache.nifi.util.NiFiProperties; -import org.apache.nifi.web.ConfigurationSnapshot; -import org.apache.nifi.web.NiFiServiceFacade; -import org.apache.nifi.web.Revision; -import org.apache.nifi.web.api.dto.RevisionDTO; -import org.apache.nifi.web.api.request.ClientIdParameter; -import org.apache.nifi.web.api.request.LongParameter; -import org.apache.commons.lang3.StringUtils; -import org.apache.nifi.controller.ScheduledState; -import org.apache.nifi.controller.service.ControllerServiceState; -import org.apache.nifi.ui.extension.UiExtension; -import org.apache.nifi.ui.extension.UiExtensionMapping; -import org.apache.nifi.web.UiExtensionType; -import static org.apache.nifi.web.api.ApplicationResource.CLIENT_ID; -import org.apache.nifi.web.api.dto.ControllerServiceDTO; -import org.apache.nifi.web.api.dto.ControllerServiceReferencingComponentDTO; -import org.apache.nifi.web.api.dto.PropertyDescriptorDTO; -import org.apache.nifi.web.api.entity.ControllerServiceEntity; -import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentsEntity; -import org.apache.nifi.web.api.entity.ControllerServicesEntity; -import org.apache.nifi.web.api.entity.PropertyDescriptorEntity; -import org.apache.nifi.web.util.Availability; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.springframework.security.access.prepost.PreAuthorize; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; /** * RESTful endpoint for managing a Controller Service. @@ -537,6 +539,170 @@ public class ControllerServiceResource extends ApplicationResource { return clusterContext(generateOkResponse(entity)).build(); } + /** + * Gets the state for a controller service. + * + * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response. + * @param availability Whether the controller service is available on the + * NCM only (ncm) or on the nodes only (node). If this instance is not + * clustered all services should use the node availability. + * @param id The id of the controller service + * @return a componentStateEntity + */ + @GET + @Consumes(MediaType.WILDCARD) + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}/{id}/state") + @PreAuthorize("hasAnyRole('ROLE_DFM')") + @ApiOperation( + value = "Gets the state for a controller service", + response = ComponentStateDTO.class, + authorizations = { + @Authorization(value = "Data Flow Manager", type = "ROLE_DFM") + } + ) + @ApiResponses( + value = { + @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(code = 401, message = "Client could not be authenticated."), + @ApiResponse(code = 403, message = "Client is not authorized to make this request."), + @ApiResponse(code = 404, message = "The specified resource could not be found."), + @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.") + } + ) + public Response getState( + @ApiParam( + value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.", + required = false + ) + @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @ApiParam( + value = "Whether the controller service is available on the NCM or nodes. If the NiFi is standalone the availability should be NODE.", + allowableValues = "NCM, NODE", + required = true + ) + @PathParam("availability") String availability, + @ApiParam( + value = "The controller service id.", + required = true + ) + @PathParam("id") String id) { + + final Availability avail = parseAvailability(availability); + + // replicate if cluster manager + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // get the component state + final ComponentStateDTO state = serviceFacade.getControllerServiceState(id); + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + + // generate the response entity + final ComponentStateEntity entity = new ComponentStateEntity(); + entity.setRevision(revision); + entity.setComponentState(state); + + // generate the response + return clusterContext(generateOkResponse(entity)).build(); + } + + /** + * Clears the state for a controller service. + * + * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response. + * @param version The revision is used to verify the client is working with the latest version of the flow. + * @param availability Whether the controller service is available on the + * NCM only (ncm) or on the nodes only (node). If this instance is not + * clustered all services should use the node availability. + * @param id The id of the controller service + * @return a componentStateEntity + */ + @POST + @Consumes(MediaType.APPLICATION_FORM_URLENCODED) + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}/{id}/state/clear-requests") + @PreAuthorize("hasAnyRole('ROLE_DFM')") + @ApiOperation( + value = "Clears the state for a controller service", + response = ComponentStateDTO.class, + authorizations = { + @Authorization(value = "Data Flow Manager", type = "ROLE_DFM") + } + ) + @ApiResponses( + value = { + @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(code = 401, message = "Client could not be authenticated."), + @ApiResponse(code = 403, message = "Client is not authorized to make this request."), + @ApiResponse(code = 404, message = "The specified resource could not be found."), + @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.") + } + ) + public Response clearState( + @Context HttpServletRequest httpServletRequest, + @ApiParam( + value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.", + required = false + ) + @FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @ApiParam( + value = "The revision is used to verify the client is working with the latest version of the flow.", + required = true + ) + @FormParam(VERSION) LongParameter version, + @ApiParam( + value = "Whether the controller service is available on the NCM or nodes. If the NiFi is standalone the availability should be NODE.", + allowableValues = "NCM, NODE", + required = true + ) + @PathParam("availability") String availability, + @ApiParam( + value = "The controller service id.", + required = true + ) + @PathParam("id") String id) { + + final Availability avail = parseAvailability(availability); + + // replicate if cluster manager + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // handle expects request (usually from the cluster manager) + final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER); + if (expects != null) { + serviceFacade.verifyCanClearControllerServiceState(id); + return generateContinueResponse().build(); + } + + // get the revision specified by the user + Long revision = null; + if (version != null) { + revision = version.getLong(); + } + + // get the component state + final ConfigurationSnapshot snapshot = serviceFacade.clearControllerServiceState(new Revision(revision, clientId.getClientId()), id); + + // create the revision + final RevisionDTO revisionDTO = new RevisionDTO(); + revisionDTO.setClientId(clientId.getClientId()); + revisionDTO.setVersion(snapshot.getVersion()); + + // generate the response entity + final ComponentStateEntity entity = new ComponentStateEntity(); + entity.setRevision(revisionDTO); + + // generate the response + return clusterContext(generateOkResponse(entity)).build(); + } + /** * Retrieves the references of the specified controller service. * diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java index 84ad8d1798..9fb90b5417 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java @@ -62,11 +62,14 @@ import org.apache.nifi.web.IllegalClusterResourceRequestException; import org.apache.nifi.web.NiFiServiceFacade; import org.apache.nifi.web.Revision; import static org.apache.nifi.web.api.ApplicationResource.CLIENT_ID; + +import org.apache.nifi.web.api.dto.ComponentStateDTO; import org.apache.nifi.web.api.dto.PositionDTO; import org.apache.nifi.web.api.dto.ProcessorConfigDTO; import org.apache.nifi.web.api.dto.ProcessorDTO; import org.apache.nifi.web.api.dto.RevisionDTO; import org.apache.nifi.web.api.dto.status.StatusHistoryDTO; +import org.apache.nifi.web.api.entity.ComponentStateEntity; import org.apache.nifi.web.api.entity.ProcessorEntity; import org.apache.nifi.web.api.entity.ProcessorsEntity; import org.apache.nifi.web.api.entity.StatusHistoryEntity; @@ -557,6 +560,149 @@ public class ProcessorResource extends ApplicationResource { return clusterContext(generateOkResponse(entity)).build(); } + /** + * Gets the state for a processor. + * + * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response. + * @param id The id of the processor + * @return a componentStateEntity + */ + @GET + @Consumes(MediaType.WILDCARD) + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{id}/state") + @PreAuthorize("hasAnyRole('ROLE_DFM')") + @ApiOperation( + value = "Gets the state for a processor", + response = ComponentStateDTO.class, + authorizations = { + @Authorization(value = "Data Flow Manager", type = "ROLE_DFM") + } + ) + @ApiResponses( + value = { + @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(code = 401, message = "Client could not be authenticated."), + @ApiResponse(code = 403, message = "Client is not authorized to make this request."), + @ApiResponse(code = 404, message = "The specified resource could not be found."), + @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.") + } + ) + public Response getState( + @ApiParam( + value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.", + required = false + ) + @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @ApiParam( + value = "The processor id.", + required = true + ) + @PathParam("id") String id) { + + // replicate if cluster manager + if (properties.isClusterManager()) { + return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // get the component state + final ComponentStateDTO state = serviceFacade.getProcessorState(groupId, id); + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + + // generate the response entity + final ComponentStateEntity entity = new ComponentStateEntity(); + entity.setRevision(revision); + entity.setComponentState(state); + + // generate the response + return clusterContext(generateOkResponse(entity)).build(); + } + + /** + * Clears the state for a processor. + * + * @param httpServletRequest servlet request + * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response. + * @param version The revision is used to verify the client is working with the latest version of the flow. + * @param id The id of the processor + * @return a componentStateEntity + */ + @POST + @Consumes(MediaType.APPLICATION_FORM_URLENCODED) + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{id}/state/clear-requests") + @PreAuthorize("hasAnyRole('ROLE_DFM')") + @ApiOperation( + value = "Clears the state for a processor", + response = ComponentStateDTO.class, + authorizations = { + @Authorization(value = "Data Flow Manager", type = "ROLE_DFM") + } + ) + @ApiResponses( + value = { + @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(code = 401, message = "Client could not be authenticated."), + @ApiResponse(code = 403, message = "Client is not authorized to make this request."), + @ApiResponse(code = 404, message = "The specified resource could not be found."), + @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.") + } + ) + public Response clearState( + @Context HttpServletRequest httpServletRequest, + @ApiParam( + value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.", + required = false + ) + @FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @ApiParam( + value = "The revision is used to verify the client is working with the latest version of the flow.", + required = true + ) + @FormParam(VERSION) LongParameter version, + @ApiParam( + value = "The processor id.", + required = true + ) + @PathParam("id") String id) { + + // replicate if cluster manager + if (properties.isClusterManager()) { + return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // handle expects request (usually from the cluster manager) + final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER); + if (expects != null) { + serviceFacade.verifyCanClearProcessorState(groupId, id); + return generateContinueResponse().build(); + } + + // get the revision specified by the user + Long revision = null; + if (version != null) { + revision = version.getLong(); + } + + // get the component state + final ConfigurationSnapshot snapshot = serviceFacade.clearProcessorState(new Revision(revision, clientId.getClientId()), groupId, id); + + // create the revision + final RevisionDTO revisionDTO = new RevisionDTO(); + revisionDTO.setClientId(clientId.getClientId()); + revisionDTO.setVersion(snapshot.getVersion()); + + // generate the response entity + final ComponentStateEntity entity = new ComponentStateEntity(); + entity.setRevision(revisionDTO); + + // generate the response + return clusterContext(generateOkResponse(entity)).build(); + } + /** * Updates the specified processor with the specified values. * diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ReportingTaskResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ReportingTaskResource.java index 1638d8ce5d..d7b77b2097 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ReportingTaskResource.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ReportingTaskResource.java @@ -54,7 +54,9 @@ import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.web.ConfigurationSnapshot; import org.apache.nifi.web.NiFiServiceFacade; import org.apache.nifi.web.Revision; +import org.apache.nifi.web.api.dto.ComponentStateDTO; import org.apache.nifi.web.api.dto.RevisionDTO; +import org.apache.nifi.web.api.entity.ComponentStateEntity; import org.apache.nifi.web.api.request.ClientIdParameter; import org.apache.nifi.web.api.request.LongParameter; import org.apache.commons.lang3.StringUtils; @@ -530,6 +532,170 @@ public class ReportingTaskResource extends ApplicationResource { return clusterContext(generateOkResponse(entity)).build(); } + /** + * Gets the state for a reporting task. + * + * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response. + * @param availability Whether the reporting task is available on the + * NCM only (ncm) or on the nodes only (node). If this instance is not + * clustered all services should use the node availability. + * @param id The id of the reporting task + * @return a componentStateEntity + */ + @GET + @Consumes(MediaType.WILDCARD) + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}/{id}/state") + @PreAuthorize("hasAnyRole('ROLE_DFM')") + @ApiOperation( + value = "Gets the state for a reporting task", + response = ComponentStateDTO.class, + authorizations = { + @Authorization(value = "Data Flow Manager", type = "ROLE_DFM") + } + ) + @ApiResponses( + value = { + @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(code = 401, message = "Client could not be authenticated."), + @ApiResponse(code = 403, message = "Client is not authorized to make this request."), + @ApiResponse(code = 404, message = "The specified resource could not be found."), + @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.") + } + ) + public Response getState( + @ApiParam( + value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.", + required = false + ) + @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @ApiParam( + value = "Whether the reporting task is available on the NCM or nodes. If the NiFi is standalone the availability should be NODE.", + allowableValues = "NCM, NODE", + required = true + ) + @PathParam("availability") String availability, + @ApiParam( + value = "The reporting task id.", + required = true + ) + @PathParam("id") String id) { + + final Availability avail = parseAvailability(availability); + + // replicate if cluster manager + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // get the component state + final ComponentStateDTO state = serviceFacade.getReportingTaskState(id); + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + + // generate the response entity + final ComponentStateEntity entity = new ComponentStateEntity(); + entity.setRevision(revision); + entity.setComponentState(state); + + // generate the response + return clusterContext(generateOkResponse(entity)).build(); + } + + /** + * Clears the state for a reporting task. + * + * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response. + * @param version The revision is used to verify the client is working with the latest version of the flow. + * @param availability Whether the reporting task is available on the + * NCM only (ncm) or on the nodes only (node). If this instance is not + * clustered all services should use the node availability. + * @param id The id of the reporting task + * @return a componentStateEntity + */ + @POST + @Consumes(MediaType.APPLICATION_FORM_URLENCODED) + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}/{id}/state/clear-requests") + @PreAuthorize("hasAnyRole('ROLE_DFM')") + @ApiOperation( + value = "Clears the state for a reporting task", + response = ComponentStateDTO.class, + authorizations = { + @Authorization(value = "Data Flow Manager", type = "ROLE_DFM") + } + ) + @ApiResponses( + value = { + @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(code = 401, message = "Client could not be authenticated."), + @ApiResponse(code = 403, message = "Client is not authorized to make this request."), + @ApiResponse(code = 404, message = "The specified resource could not be found."), + @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.") + } + ) + public Response clearState( + @Context HttpServletRequest httpServletRequest, + @ApiParam( + value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.", + required = false + ) + @FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @ApiParam( + value = "The revision is used to verify the client is working with the latest version of the flow.", + required = true + ) + @FormParam(VERSION) LongParameter version, + @ApiParam( + value = "Whether the reporting task is available on the NCM or nodes. If the NiFi is standalone the availability should be NODE.", + allowableValues = "NCM, NODE", + required = true + ) + @PathParam("availability") String availability, + @ApiParam( + value = "The reporting task id.", + required = true + ) + @PathParam("id") String id) { + + final Availability avail = parseAvailability(availability); + + // replicate if cluster manager + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // handle expects request (usually from the cluster manager) + final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER); + if (expects != null) { + serviceFacade.verifyCanClearReportingTaskState(id); + return generateContinueResponse().build(); + } + + // get the revision specified by the user + Long revision = null; + if (version != null) { + revision = version.getLong(); + } + + // get the component state + final ConfigurationSnapshot snapshot = serviceFacade.clearReportingTaskState(new Revision(revision, clientId.getClientId()), id); + + // create the revision + final RevisionDTO revisionDTO = new RevisionDTO(); + revisionDTO.setClientId(clientId.getClientId()); + revisionDTO.setVersion(snapshot.getVersion()); + + // generate the response entity + final ComponentStateEntity entity = new ComponentStateEntity(); + entity.setRevision(revisionDTO); + + // generate the response + return clusterContext(generateOkResponse(entity)).build(); + } + /** * Updates the specified reporting task. * diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java index 6f7a733488..339b725692 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java @@ -16,27 +16,7 @@ */ package org.apache.nifi.web.api.dto; -import java.text.Collator; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.Comparator; -import java.util.Date; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedHashMap; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Set; -import java.util.TreeMap; -import java.util.TreeSet; -import java.util.concurrent.TimeUnit; - -import javax.ws.rs.WebApplicationException; - +import org.apache.commons.lang3.StringUtils; import org.apache.nifi.action.Action; import org.apache.nifi.action.component.details.ComponentDetails; import org.apache.nifi.action.component.details.ExtensionDetails; @@ -52,6 +32,7 @@ import org.apache.nifi.action.details.FlowChangeMoveDetails; import org.apache.nifi.action.details.FlowChangePurgeDetails; import org.apache.nifi.action.details.MoveDetails; import org.apache.nifi.action.details.PurgeDetails; +import org.apache.nifi.annotation.behavior.Stateful; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.authorization.Authority; @@ -62,25 +43,34 @@ import org.apache.nifi.cluster.protocol.NodeIdentifier; import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateMap; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.ConnectableType; import org.apache.nifi.connectable.Connection; import org.apache.nifi.connectable.Funnel; import org.apache.nifi.connectable.Port; import org.apache.nifi.connectable.Position; +import org.apache.nifi.controller.ConfiguredComponent; import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.ControllerServiceLookup; import org.apache.nifi.controller.Counter; import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.ReportingTaskNode; import org.apache.nifi.controller.Snippet; import org.apache.nifi.controller.Template; import org.apache.nifi.controller.label.Label; +import org.apache.nifi.controller.queue.DropFlowFileState; +import org.apache.nifi.controller.queue.DropFlowFileStatus; import org.apache.nifi.controller.queue.FlowFileSummary; import org.apache.nifi.controller.queue.ListFlowFileState; import org.apache.nifi.controller.queue.ListFlowFileStatus; +import org.apache.nifi.controller.queue.QueueSize; import org.apache.nifi.controller.repository.FlowFileRecord; import org.apache.nifi.controller.repository.claim.ContentClaim; import org.apache.nifi.controller.repository.claim.ResourceClaim; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.controller.service.ControllerServiceReference; import org.apache.nifi.controller.status.ConnectionStatus; import org.apache.nifi.controller.status.PortStatus; import org.apache.nifi.controller.status.ProcessGroupStatus; @@ -106,10 +96,12 @@ import org.apache.nifi.remote.RemoteGroupPort; import org.apache.nifi.remote.RootGroupPort; import org.apache.nifi.reporting.Bulletin; import org.apache.nifi.reporting.BulletinRepository; +import org.apache.nifi.reporting.ReportingTask; import org.apache.nifi.scheduling.SchedulingStrategy; import org.apache.nifi.user.NiFiUser; import org.apache.nifi.user.NiFiUserGroup; import org.apache.nifi.util.FormatUtils; +import org.apache.nifi.web.FlowModification; import org.apache.nifi.web.Revision; import org.apache.nifi.web.api.dto.PropertyDescriptorDTO.AllowableValueDTO; import org.apache.nifi.web.api.dto.action.ActionDTO; @@ -134,16 +126,26 @@ import org.apache.nifi.web.api.dto.status.ProcessGroupStatusDTO; import org.apache.nifi.web.api.dto.status.ProcessorStatusDTO; import org.apache.nifi.web.api.dto.status.RemoteProcessGroupStatusDTO; import org.apache.nifi.web.api.dto.status.StatusDTO; -import org.apache.commons.lang3.StringUtils; -import org.apache.nifi.controller.ConfiguredComponent; -import org.apache.nifi.controller.ReportingTaskNode; -import org.apache.nifi.controller.queue.DropFlowFileState; -import org.apache.nifi.controller.queue.DropFlowFileStatus; -import org.apache.nifi.controller.queue.QueueSize; -import org.apache.nifi.controller.service.ControllerServiceNode; -import org.apache.nifi.controller.service.ControllerServiceReference; -import org.apache.nifi.reporting.ReportingTask; -import org.apache.nifi.web.FlowModification; + +import javax.ws.rs.WebApplicationException; +import java.text.Collator; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.Date; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.concurrent.TimeUnit; public final class DtoFactory { @@ -270,6 +272,66 @@ public final class DtoFactory { return historyDto; } + /** + * Creates a ComponentStateDTO for the given component and state's. + * + * @param componentId component id + * @param localState local state + * @param clusterState cluster state + * @return dto + */ + public ComponentStateDTO createComponentStateDTO(final String componentId, final Class componentClass, final StateMap localState, final StateMap clusterState) { + final ComponentStateDTO dto = new ComponentStateDTO(); + dto.setComponentId(componentId); + dto.setStateDescription(getStateDescription(componentClass)); + dto.setLocalState(createStateMapDTO(Scope.LOCAL, localState)); + dto.setClusterState(createStateMapDTO(Scope.CLUSTER, clusterState)); + return dto; + } + + /** + * Gets the description of the state this component persists. + * + * @param componentClass the component class + * @return state description + */ + private String getStateDescription(final Class componentClass) { + final Stateful capabilityDesc = componentClass.getAnnotation(Stateful.class); + if (capabilityDesc != null) { + return capabilityDesc.description(); + } else { + return null; + } + } + + /** + * Creates a StateMapDTO for the given scope and state map. + * + * @param scope the scope + * @param stateMap the state map + * @return dto + */ + public StateMapDTO createStateMapDTO(final Scope scope, final StateMap stateMap) { + if (stateMap == null) { + return null; + } + + final StateMapDTO dto = new StateMapDTO(); + dto.setScope(scope.toString()); + + final List stateEntries = new ArrayList<>(); + final Map state = stateMap.toMap(); + for (final Map.Entry entry : state.entrySet()) { + final StateEntryDTO entryDTO = new StateEntryDTO(); + entryDTO.setKey(entry.getKey()); + entryDTO.setValue(entry.getValue()); + stateEntries.add(entryDTO); + } + dto.setState(stateEntries); + + return dto; + } + /** * Creates CounterDTOs for each Counter specified. * @@ -1021,6 +1083,7 @@ public final class DtoFactory { dto.setActiveThreadCount(reportingTaskNode.getActiveThreadCount()); dto.setAnnotationData(reportingTaskNode.getAnnotationData()); dto.setComments(reportingTaskNode.getComments()); + dto.setPersistsState(reportingTaskNode.getReportingTask().getClass().isAnnotationPresent(Stateful.class)); final Map defaultSchedulingPeriod = new HashMap<>(); defaultSchedulingPeriod.put(SchedulingStrategy.TIMER_DRIVEN.name(), SchedulingStrategy.TIMER_DRIVEN.getDefaultSchedulingPeriod()); @@ -1088,6 +1151,7 @@ public final class DtoFactory { dto.setState(controllerServiceNode.getState().name()); dto.setAnnotationData(controllerServiceNode.getAnnotationData()); dto.setComments(controllerServiceNode.getComments()); + dto.setPersistsState(controllerServiceNode.getControllerServiceImplementation().getClass().isAnnotationPresent(Stateful.class)); // sort a copy of the properties final Map sortedProperties = new TreeMap<>(new Comparator() { @@ -1565,6 +1629,7 @@ public final class DtoFactory { dto.setStyle(node.getStyle()); dto.setParentGroupId(node.getProcessGroup().getIdentifier()); dto.setInputRequirement(node.getInputRequirement().name()); + dto.setPersistsState(node.getProcessor().getClass().isAnnotationPresent(Stateful.class)); dto.setType(node.getProcessor().getClass().getCanonicalName()); dto.setName(node.getName()); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java index dbf24e1c08..a349f2a3e5 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java @@ -16,27 +16,6 @@ */ package org.apache.nifi.web.controller; -import java.io.IOException; -import java.io.InputStream; -import java.text.Collator; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.Comparator; -import java.util.Date; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Set; -import java.util.SortedSet; -import java.util.TimeZone; -import java.util.TreeSet; -import java.util.concurrent.TimeUnit; - -import javax.ws.rs.WebApplicationException; - import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.ClassUtils; import org.apache.commons.lang3.StringUtils; @@ -119,6 +98,26 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.security.access.AccessDeniedException; +import javax.ws.rs.WebApplicationException; +import java.io.IOException; +import java.io.InputStream; +import java.text.Collator; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.Date; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Set; +import java.util.SortedSet; +import java.util.TimeZone; +import java.util.TreeSet; +import java.util.concurrent.TimeUnit; + public class ControllerFacade { private static final Logger logger = LoggerFactory.getLogger(ControllerFacade.class); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ComponentStateDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ComponentStateDAO.java new file mode 100644 index 0000000000..636addb02f --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ComponentStateDAO.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.web.dao; + +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateMap; +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.ReportingTaskNode; +import org.apache.nifi.controller.service.ControllerServiceNode; + +public interface ComponentStateDAO { + + /** + * Gets the state map for the specified processor. + * + * @param processor processor + * @param scope scope + * @return state map + */ + StateMap getState(ProcessorNode processor, Scope scope); + + /** + * Clears the state for the specified processor. + * + * @param processor processor + */ + void clearState(ProcessorNode processor); + + /** + * Gets the state map for the specified controller service. + * + * @param controllerService controller service + * @param scope scope + * @return state map + */ + StateMap getState(ControllerServiceNode controllerService, Scope scope); + + /** + * Clears the state for the specified controller service. + * + * @param controllerService controller service + */ + void clearState(ControllerServiceNode controllerService); + + /** + * Gets the state for the specified reporting task. + * + * @param reportingTask reporting task + * @param scope scope + * @return state map + */ + StateMap getState(ReportingTaskNode reportingTask, Scope scope); + + /** + * Clears the state for the specified reporting task. + * + * @param reportingTask reporting task + */ + void clearState(ReportingTaskNode reportingTask); +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ControllerServiceDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ControllerServiceDAO.java index bb18b1b167..033634b663 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ControllerServiceDAO.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ControllerServiceDAO.java @@ -17,6 +17,9 @@ package org.apache.nifi.web.dao; import java.util.Set; + +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateMap; import org.apache.nifi.controller.ScheduledState; import org.apache.nifi.controller.service.ControllerServiceNode; @@ -102,4 +105,26 @@ public interface ControllerServiceDAO { * @param controllerServiceId The controller service id */ void deleteControllerService(String controllerServiceId); + + /** + * Gets the specified controller service. + * + * @param controllerServiceId controller service id + * @return state map + */ + StateMap getState(String controllerServiceId, Scope scope); + + /** + * Verifies the controller service can clear state. + * + * @param controllerServiceId controller service id + */ + void verifyClearState(String controllerServiceId); + + /** + * Clears the state of the specified controller service. + * + * @param controllerServiceId controller service id + */ + void clearState(String controllerServiceId); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessorDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessorDAO.java index 30cce5da18..9f45c90846 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessorDAO.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessorDAO.java @@ -17,6 +17,9 @@ package org.apache.nifi.web.dao; import java.util.Set; + +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateMap; import org.apache.nifi.controller.ProcessorNode; import org.apache.nifi.web.api.dto.ProcessorDTO; @@ -88,4 +91,28 @@ public interface ProcessorDAO { */ void deleteProcessor(String groupId, String processorId); + /** + * Gets the specified processor. + * + * @param groupId group id + * @param processorId processor id + * @return state map + */ + StateMap getState(String groupId, String processorId, Scope scope); + + /** + * Verifies the processor can clear state. + * + * @param groupId group id + * @param processorId processor id + */ + void verifyClearState(String groupId, String processorId); + + /** + * Clears the state of the specified processor. + * + * @param groupId group id + * @param processorId processor id + */ + void clearState(String groupId, String processorId); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ReportingTaskDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ReportingTaskDAO.java index 43852c24d0..ab1a130c79 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ReportingTaskDAO.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ReportingTaskDAO.java @@ -17,6 +17,9 @@ package org.apache.nifi.web.dao; import java.util.Set; + +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateMap; import org.apache.nifi.controller.ReportingTaskNode; import org.apache.nifi.web.api.dto.ReportingTaskDTO; @@ -82,4 +85,26 @@ public interface ReportingTaskDAO { * @param reportingTaskId The reporting task id */ void deleteReportingTask(String reportingTaskId); + + /** + * Gets the specified reporting task. + * + * @param reportingTaskId reporting task id + * @return state map + */ + StateMap getState(String reportingTaskId, Scope scope); + + /** + * Verifies the reporting task can clear state. + * + * @param reportingTaskId reporting task id + */ + void verifyClearState(String reportingTaskId); + + /** + * Clears the state of the specified reporting task. + * + * @param reportingTaskId reporting task id + */ + void clearState(String reportingTaskId); } 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/StandardComponentStateDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardComponentStateDAO.java new file mode 100644 index 0000000000..f0a9094daf --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardComponentStateDAO.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.web.dao.impl; + +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.StateMap; +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.ReportingTaskNode; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.web.ResourceNotFoundException; +import org.apache.nifi.web.dao.ComponentStateDAO; + +import java.io.IOException; + +public class StandardComponentStateDAO implements ComponentStateDAO { + + private StateManagerProvider stateManagerProvider; + + private StateMap getState(final String componentId, final Scope scope) { + try { + final StateManager manager = stateManagerProvider.getStateManager(componentId); + if (manager == null) { + throw new ResourceNotFoundException(String.format("State for the specified component %s could not be found.", componentId)); + } + + return manager.getState(scope); + } catch (final IOException ioe) { + throw new IllegalStateException(String.format("Unable to get the state for the specified component %s: %s", componentId, ioe), ioe); + } + } + + private void clearState(final String componentId) { + try { + final StateManager manager = stateManagerProvider.getStateManager(componentId); + if (manager == null) { + throw new ResourceNotFoundException(String.format("State for the specified component %s could not be found.", componentId)); + } + + // clear both state's at the same time + manager.clear(Scope.CLUSTER); + manager.clear(Scope.LOCAL); + } catch (final IOException ioe) { + throw new IllegalStateException(String.format("Unable to clear the state for the specified component %s: %s", componentId, ioe), ioe); + } + } + + @Override + public StateMap getState(ProcessorNode processor, Scope scope) { + return getState(processor.getIdentifier(), scope); + } + + @Override + public void clearState(ProcessorNode processor) { + clearState(processor.getIdentifier()); + } + + @Override + public StateMap getState(ControllerServiceNode controllerService, Scope scope) { + return getState(controllerService.getIdentifier(), scope); + } + + @Override + public void clearState(ControllerServiceNode controllerService) { + clearState(controllerService.getIdentifier()); + } + + @Override + public StateMap getState(ReportingTaskNode reportingTask, Scope scope) { + return getState(reportingTask.getIdentifier(), scope); + } + + @Override + public void clearState(ReportingTaskNode reportingTask) { + clearState(reportingTask.getIdentifier()); + } + + /* setters */ + + public void setStateManagerProvider(StateManagerProvider stateManagerProvider) { + this.stateManagerProvider = stateManagerProvider; + } +} 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..bcace0ce6a 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 @@ -20,6 +20,9 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Set; + +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateMap; import org.apache.nifi.controller.ScheduledState; import org.apache.nifi.controller.exception.ControllerServiceInstantiationException; @@ -31,11 +34,13 @@ import org.apache.nifi.controller.service.ControllerServiceState; import org.apache.nifi.web.NiFiCoreException; import org.apache.nifi.web.ResourceNotFoundException; import org.apache.nifi.web.api.dto.ControllerServiceDTO; +import org.apache.nifi.web.dao.ComponentStateDAO; import org.apache.nifi.web.dao.ControllerServiceDAO; public class StandardControllerServiceDAO extends ComponentDAO implements ControllerServiceDAO { private ControllerServiceProvider serviceProvider; + private ComponentStateDAO componentStateDAO; private ControllerServiceNode locateControllerService(final String controllerServiceId) { // get the controller service @@ -255,8 +260,30 @@ public class StandardControllerServiceDAO extends ComponentDAO implements Contro serviceProvider.removeControllerService(controllerService); } + @Override + public StateMap getState(String controllerServiceId, Scope scope) { + final ControllerServiceNode controllerService = locateControllerService(controllerServiceId); + return componentStateDAO.getState(controllerService, scope); + } + + @Override + public void verifyClearState(String controllerServiceId) { + final ControllerServiceNode controllerService = locateControllerService(controllerServiceId); + controllerService.verifyCanClearState(); + } + + @Override + public void clearState(String controllerServiceId) { + final ControllerServiceNode controllerService = locateControllerService(controllerServiceId); + componentStateDAO.clearState(controllerService); + } + /* setters */ public void setServiceProvider(ControllerServiceProvider serviceProvider) { this.serviceProvider = serviceProvider; } + + public void setComponentStateDAO(ComponentStateDAO componentStateDAO) { + this.componentStateDAO = componentStateDAO; + } } 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..48d2bee030 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 @@ -26,6 +26,8 @@ import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; import java.util.regex.Matcher; +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateMap; import org.apache.nifi.connectable.Connection; import org.apache.nifi.connectable.Position; import org.apache.nifi.controller.FlowController; @@ -43,6 +45,7 @@ import org.apache.nifi.web.NiFiCoreException; import org.apache.nifi.web.ResourceNotFoundException; import org.apache.nifi.web.api.dto.ProcessorConfigDTO; import org.apache.nifi.web.api.dto.ProcessorDTO; +import org.apache.nifi.web.dao.ComponentStateDAO; import org.apache.nifi.web.dao.ProcessorDAO; import org.apache.commons.lang3.StringUtils; @@ -54,6 +57,7 @@ public class StandardProcessorDAO extends ComponentDAO implements ProcessorDAO { private static final Logger logger = LoggerFactory.getLogger(StandardProcessorDAO.class); private FlowController flowController; + private ComponentStateDAO componentStateDAO; private ProcessorNode locateProcessor(String groupId, String processorId) { return locateProcessor(locateProcessGroup(flowController, groupId), processorId); @@ -452,8 +456,30 @@ public class StandardProcessorDAO extends ComponentDAO implements ProcessorDAO { } } + @Override + public StateMap getState(String groupId, String processorId, final Scope scope) { + final ProcessorNode processor = locateProcessor(groupId, processorId); + return componentStateDAO.getState(processor, scope); + } + + @Override + public void verifyClearState(String groupId, String processorId) { + final ProcessorNode processor = locateProcessor(groupId, processorId); + processor.verifyCanClearState(); + } + + @Override + public void clearState(String groupId, String processorId) { + final ProcessorNode processor = locateProcessor(groupId, processorId); + componentStateDAO.clearState(processor); + } + /* setters */ public void setFlowController(FlowController flowController) { this.flowController = flowController; } + + public void setComponentStateDAO(ComponentStateDAO componentStateDAO) { + this.componentStateDAO = componentStateDAO; + } } 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..e3cf89bd75 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 @@ -24,6 +24,8 @@ import java.util.Set; import java.util.concurrent.RejectedExecutionException; import java.util.regex.Matcher; import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateMap; import org.apache.nifi.controller.ReportingTaskNode; import org.apache.nifi.controller.ScheduledState; import org.apache.nifi.controller.exception.ComponentLifeCycleException; @@ -36,12 +38,14 @@ import org.apache.nifi.util.FormatUtils; import org.apache.nifi.web.NiFiCoreException; import org.apache.nifi.web.ResourceNotFoundException; import org.apache.nifi.web.api.dto.ReportingTaskDTO; +import org.apache.nifi.web.dao.ComponentStateDAO; import org.apache.nifi.web.dao.ReportingTaskDAO; import org.quartz.CronExpression; public class StandardReportingTaskDAO extends ComponentDAO implements ReportingTaskDAO { private ReportingTaskProvider reportingTaskProvider; + private ComponentStateDAO componentStateDAO; private ReportingTaskNode locateReportingTask(final String reportingTaskId) { // get the reporting task @@ -299,8 +303,30 @@ public class StandardReportingTaskDAO extends ComponentDAO implements ReportingT reportingTaskProvider.removeReportingTask(reportingTask); } + @Override + public StateMap getState(String reportingTaskId, Scope scope) { + final ReportingTaskNode reportingTask = locateReportingTask(reportingTaskId); + return componentStateDAO.getState(reportingTask, scope); + } + + @Override + public void verifyClearState(String reportingTaskId) { + final ReportingTaskNode reportingTask = locateReportingTask(reportingTaskId); + reportingTask.verifyCanClearState(); + } + + @Override + public void clearState(String reportingTaskId) { + final ReportingTaskNode reportingTask = locateReportingTask(reportingTaskId); + componentStateDAO.clearState(reportingTask); + } + /* setters */ public void setReportingTaskProvider(ReportingTaskProvider reportingTaskProvider) { this.reportingTaskProvider = reportingTaskProvider; } + + public void setComponentStateDAO(ComponentStateDAO componentStateDAO) { + this.componentStateDAO = componentStateDAO; + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/StateManagerProviderFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/StateManagerProviderFactoryBean.java new file mode 100644 index 0000000000..8d4e37ea23 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/StateManagerProviderFactoryBean.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.web.spring; + +import org.apache.nifi.cluster.manager.impl.WebClusterManager; +import org.apache.nifi.components.state.StateManagerProvider; +import org.apache.nifi.controller.FlowController; +import org.apache.nifi.util.NiFiProperties; +import org.springframework.beans.BeansException; +import org.springframework.beans.factory.FactoryBean; +import org.springframework.context.ApplicationContext; +import org.springframework.context.ApplicationContextAware; + +/** + * + */ +public class StateManagerProviderFactoryBean implements FactoryBean, ApplicationContextAware { + + private ApplicationContext context; + private StateManagerProvider stateManagerProvider; + private NiFiProperties properties; + + @Override + public Object getObject() throws Exception { + if (stateManagerProvider == null) { + if (properties.isClusterManager()) { + final WebClusterManager webClusterManager = context.getBean("clusterManager", WebClusterManager.class); + if (webClusterManager != null) { + stateManagerProvider = webClusterManager.getStateManagerProvider(); + } + } else { + final FlowController flowController = context.getBean("flowController", FlowController.class); + if (flowController != null) { + stateManagerProvider = flowController.getStateManagerProvider(); + } + } + } + + return stateManagerProvider; + } + + @Override + public Class getObjectType() { + return StateManagerProvider.class; + } + + @Override + public boolean isSingleton() { + return true; + } + + public void setProperties(NiFiProperties properties) { + this.properties = properties; + } + + @Override + public void setApplicationContext(ApplicationContext context) throws BeansException { + this.context = context; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml index 406f38cf7f..b38bca5fdb 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml @@ -31,7 +31,12 @@ - + + + + + + @@ -80,12 +85,18 @@ + + + + + + @@ -358,6 +369,11 @@ + + + + + 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-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml index 01a1ccb8e8..b057dd3e72 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml @@ -282,6 +282,7 @@ ${staging.dir}/js/nf/canvas/nf-canvas-toolbox.js ${staging.dir}/js/nf/canvas/nf-custom-ui.js ${staging.dir}/js/nf/canvas/nf-queue-listing.js + ${staging.dir}/js/nf/canvas/nf-component-state.js ${staging.dir}/js/nf/canvas/nf-controller-service.js ${staging.dir}/js/nf/canvas/nf-reporting-task.js ${staging.dir}/js/nf/canvas/nf-processor-configuration.js @@ -457,6 +458,7 @@ ${staging.dir}/css/port-configuration.css ${staging.dir}/css/port-details.css ${staging.dir}/css/queue-listing.css + ${staging.dir}/css/component-state.css ${staging.dir}/css/label-configuration.css ${staging.dir}/css/connection-configuration.css ${staging.dir}/css/connection-details.css diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/filters/canvas.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/filters/canvas.properties index 289d94eaa4..e626859d9e 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/filters/canvas.properties +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/filters/canvas.properties @@ -23,6 +23,7 @@ nf.canvas.script.tags=\n\ \n\ \n\ +\n\ \n\ \n\ \n\ diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/canvas.jsp b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/canvas.jsp index 5a9eab7421..287a58c481 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/canvas.jsp +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/canvas.jsp @@ -122,6 +122,7 @@ +
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/component-state-dialog.jsp b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/component-state-dialog.jsp new file mode 100644 index 0000000000..abc1140fb6 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/component-state-dialog.jsp @@ -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. +--%> +<%@ page contentType="text/html" pageEncoding="UTF-8" session="false" %> +
+
+
+
Name
+
+
+
+
+
+
Description
+
+
+
+
+
+
+ +
+
+ Displaying  of  +
+
+
+ +
+
\ No newline at end of file diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/canvas.css b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/canvas.css index 2df31d6f0c..1a4eb66142 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/canvas.css +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/canvas.css @@ -22,6 +22,7 @@ @import url(queue-listing.css); @import url(remote-process-group-configuration.css); @import url(controller-service.css); +@import url(component-state.css); @import url(reporting-task.css); @import url(port-configuration.css); @import url(port-details.css); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/component-state.css b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/component-state.css new file mode 100644 index 0000000000..34797b1d41 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/component-state.css @@ -0,0 +1,92 @@ +/* + * 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. + */ + +/* + Component state +*/ + +#component-state-dialog { + position: absolute; + overflow: hidden; + width: 600px; + font-size: 10px; + z-index: 1301; + display: none; +} + +#component-state-description { + width: 580px; + height: 50px; + font-size: 10px; + line-height: 12px; +} + +/* + Component state filter +*/ + +#component-state-filter-controls { + float: right; + margin-top: 10px; + margin-right: 2px; + margin-bottom: 7px; +} + +#component-state-filter-status { + font-size: 9px; + font-weight: bold; + color: #9f6000; + clear: left; + line-height: normal; + margin-left: 5px; +} + +#component-state-filter { + padding: 3px 0px 1px 3px; + font-size: 12px; + height: 18px; + line-height: 20px; + width: 173px; + border: 1px solid #ccc; + float: left; +} + +/* + Component state table +*/ + +#component-state-table { + width: 578px; + height: 235px; + border: 1px solid #666; +} + +/* + Clear +*/ + +#clear-link-container { + margin-top: 16px; + margin-bottom: 18px; +} + +#clear-link.disabled { + color: #bbb; + font-style: italic; + text-decoration: none !important; + cursor: default !important; +} \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/new-controller-service-dialog.css b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/new-controller-service-dialog.css index 57c3c555e7..6d479a9bd2 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/new-controller-service-dialog.css +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/new-controller-service-dialog.css @@ -49,13 +49,15 @@ font-size: 12pt; font-weight: bold; color: #527991; - margin-bottom: 8px; + margin-bottom: 6px; width: 588px; } #controller-service-type-description { width: 588px; - height: 60px; + height: 62px; + font-size: 10px; + line-height: 12px; } #controller-service-availability-container { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/new-processor-dialog.css b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/new-processor-dialog.css index 9ccd23e28e..21dd2921ba 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/new-processor-dialog.css +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/new-processor-dialog.css @@ -48,13 +48,15 @@ font-size: 12pt; font-weight: bold; color: #527991; - margin-bottom: 8px; + margin-bottom: 6px; width: 588px; } #processor-type-description { width: 588px; - height: 60px; + height: 62px; + font-size: 10px; + line-height: 12px; } #processor-types-table { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/new-reporting-task-dialog.css b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/new-reporting-task-dialog.css index c8d629f50a..88a4e494a2 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/new-reporting-task-dialog.css +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/new-reporting-task-dialog.css @@ -49,13 +49,15 @@ font-size: 12pt; font-weight: bold; color: #527991; - margin-bottom: 8px; + margin-bottom: 6px; width: 588px; } #reporting-task-type-description { width: 588px; - height: 60px; + height: 62px; + font-size: 10px; + line-height: 12px; } #reporting-task-availability-container { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconViewState.png b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconViewState.png new file mode 100644 index 0000000000..c9ffb383ac Binary files /dev/null and b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconViewState.png differ diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-actions.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-actions.js index 5343323d18..c0e4e619be 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-actions.js +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-actions.js @@ -1063,6 +1063,23 @@ nf.Actions = (function () { nf.QueueListing.listQueue(connection); }, + /** + * Views the state for the specified processor. + * + * @param {selection} selection + */ + viewState: function (selection) { + if (selection.size() !== 1 || !nf.CanvasUtils.isProcessor(selection)) { + return; + } + + // get the processor data + var processor = selection.datum(); + + // view the state for the selected processor + nf.ComponentState.showState(processor.component, nf.CanvasUtils.supportsModification(selection)); + }, + /** * Opens the fill color dialog for the component in the specified selection. * diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.js index dd9df5a995..e11aa5669a 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.js +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.js @@ -1080,6 +1080,7 @@ nf.Canvas = (function () { nf.Settings.init(); nf.Actions.init(); nf.QueueListing.init(); + nf.ComponentState.init(); // initialize the component behaviors nf.Draggable.init(); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-component-state.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-component-state.js new file mode 100644 index 0000000000..769256c239 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-component-state.js @@ -0,0 +1,376 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* global nf */ + +/** + * Views state for a given component. + */ +nf.ComponentState = (function () { + + var config = { + filterText: 'Filter', + styles: { + filterList: 'filter-list' + } + }; + + /** + * Filters the component state table. + */ + var applyFilter = function () { + // get the dataview + var componentStateTable = $('#component-state-table').data('gridInstance'); + + // ensure the grid has been initialized + if (nf.Common.isDefinedAndNotNull(componentStateTable)) { + var componentStateData = componentStateTable.getData(); + + // update the search criteria + componentStateData.setFilterArgs({ + searchString: getFilterText() + }); + componentStateData.refresh(); + } + }; + + /** + * Determines if the item matches the filter. + * + * @param {object} item The item to filter + * @param {object} args The filter criteria + * @returns {boolean} Whether the item matches the filter + */ + var filter = function (item, args) { + if (args.searchString === '') { + return true; + } + + try { + // perform the row filtering + var filterExp = new RegExp(args.searchString, 'i'); + } catch (e) { + // invalid regex + return false; + } + + // determine if the item matches the filter + var matchesKey = item['key'].search(filterExp) >= 0; + var matchesValue = item['value'].search(filterExp) >= 0; + + // conditionally consider the scope + var matchesScope = false; + if (nf.Common.isDefinedAndNotNull(item['scope'])) { + matchesScope = item['scope'].search(filterExp) >= 0; + } + + return matchesKey || matchesValue || matchesScope; + }; + + /** + * Sorts the specified data using the specified sort details. + * + * @param {object} sortDetails + * @param {object} data + */ + var sort = function (sortDetails, data) { + // defines a function for sorting + var comparer = function (a, b) { + var aString = nf.Common.isDefinedAndNotNull(a[sortDetails.columnId]) ? a[sortDetails.columnId] : ''; + var bString = nf.Common.isDefinedAndNotNull(b[sortDetails.columnId]) ? b[sortDetails.columnId] : ''; + return aString === bString ? 0 : aString > bString ? 1 : -1; + }; + + // perform the sort + data.sort(comparer, sortDetails.sortAsc); + }; + + /** + * Get the text out of the filter field. If the filter field doesn't + * have any text it will contain the text 'filter list' so this method + * accounts for that. + */ + var getFilterText = function () { + var filterText = ''; + var filterField = $('#component-state-filter'); + if (!filterField.hasClass(config.styles.filterList)) { + filterText = filterField.val(); + } + return filterText; + }; + + /** + * Clears the component state table. + */ + var clearTable = function () { + var componentStateGrid = $('#component-state-table').data('gridInstance'); + var componentStateData = componentStateGrid.getData(); + componentStateData.setItems([]); + + // clear the total number entries + $('#displayed-component-state-entries').text('0'); + $('#total-component-state-entries').text('0'); + }; + + /** + * Loads the table with the component state. + * + * @param {object} componentState + */ + var loadComponentState = function (localState, clusterState) { + var count = 0; + + var componentStateGrid = $('#component-state-table').data('gridInstance'); + var componentStateData = componentStateGrid.getData(); + + // begin the update + componentStateData.beginUpdate(); + + // local state + if (nf.Common.isDefinedAndNotNull(localState)) { + $.each(localState.state, function (i, stateEntry) { + componentStateData.addItem($.extend({ + id: count++, + scope: stateEntry.clusterNodeAddress + }, stateEntry)); + }); + } + + if (nf.Common.isDefinedAndNotNull(clusterState)) { + $.each(clusterState.state, function (i, stateEntry) { + componentStateData.addItem($.extend({ + id: count++, + scope: 'Cluster' + }, stateEntry)); + }); + } + + // complete the update + componentStateData.endUpdate(); + componentStateData.reSort(); + + // update the total number of state entries + $('#total-component-state-entries').text(count); + }; + + /** + * Reset the dialog. + */ + var resetDialog = function () { + // clear the fields + $('#component-state-name').text(''); + $('#component-state-description').text(''); + + // clear any filter strings + $('#component-state-filter').addClass(config.styles.filterList).val(config.filterText); + + // reset clear link + $('#clear-link').removeClass('disabled').attr('title', ''); + + // clear the table + clearTable(); + + // clear the component + $('#component-state-table').removeData('component'); + }; + + return { + init: function () { + // intialize the component state filter + $('#component-state-filter').on('focus', function () { + if ($(this).hasClass(config.styles.filterList)) { + $(this).removeClass(config.styles.filterList).val(''); + } + }).on('blur', function () { + if ($(this).val() === '') { + $(this).addClass(config.styles.filterList).val(config.filterText); + } + }).on('keyup', function () { + applyFilter(); + }).addClass(config.styles.filterList).val(config.filterText); + + // initialize the processor configuration dialog + $('#component-state-dialog').modal({ + headerText: 'Component State', + overlayBackground: false, + buttons: [{ + buttonText: 'Close', + handler: { + click: function () { + $(this).modal('hide'); + } + } + }], + handler: { + close: function () { + resetDialog(); + } + } + }).draggable({ + containment: 'parent', + handle: '.dialog-header' + }); + + // clear state link + $('#clear-link').on('click', function () { + if ($(this).hasClass('disabled') === false) { + var componentStateTable = $('#component-state-table'); + + // ensure there is state to clear + var componentStateGrid = componentStateTable.data('gridInstance'); + var stateEntryCount = componentStateGrid.getDataLength(); + + if (stateEntryCount > 0) { + // clear the state + var revision = nf.Client.getRevision(); + var component = componentStateTable.data('component'); + $.ajax({ + type: 'POST', + url: component.uri + '/state/clear-requests', + data: { + version: revision.version, + clientId: revision.clientId + }, + dataType: 'json' + }).done(function (response) { + // update the revision + nf.Client.setRevision(response.revision); + + // clear the table + clearTable(); + + // reload the table with no state + loadComponentState() + }).fail(nf.Common.handleAjaxError); + } else { + nf.Dialog.showOkDialog({ + dialogContent: 'This component has no state to clear.', + overlayBackground: false + }); + } + } + }); + + // initialize the queue listing table + var componentStateColumns = [ + {id: 'key', field: 'key', name: 'Key', sortable: true, resizable: true}, + {id: 'value', field: 'value', name: 'Value', sortable: true, resizable: true} + ]; + + // conditionally show the cluster node identifier + if (nf.Canvas.isClustered()) { + componentStateColumns.push({id: 'scope', field: 'scope', name: 'Scope', sortable: true, resizable: true}); + } + + var componentStateOptions = { + forceFitColumns: true, + enableTextSelectionOnCells: true, + enableCellNavigation: false, + enableColumnReorder: false, + autoEdit: false + }; + + // initialize the dataview + var componentStateData = new Slick.Data.DataView({ + inlineFilters: false + }); + componentStateData.setItems([]); + componentStateData.setFilterArgs({ + searchString: '', + property: 'key' + }); + componentStateData.setFilter(filter); + + // initialize the sort + sort({ + columnId: 'key', + sortAsc: true + }, componentStateData); + + // initialize the grid + var componentStateGrid = new Slick.Grid('#component-state-table', componentStateData, componentStateColumns, componentStateOptions); + componentStateGrid.setSelectionModel(new Slick.RowSelectionModel()); + componentStateGrid.registerPlugin(new Slick.AutoTooltips()); + componentStateGrid.setSortColumn('key', true); + componentStateGrid.onSort.subscribe(function (e, args) { + sort({ + columnId: args.sortCol.field, + sortAsc: args.sortAsc + }, componentStateData); + }); + + // wire up the dataview to the grid + componentStateData.onRowCountChanged.subscribe(function (e, args) { + componentStateGrid.updateRowCount(); + componentStateGrid.render(); + + // update the total number of displayed items + $('#displayed-component-state-entries').text(args.current); + }); + componentStateData.onRowsChanged.subscribe(function (e, args) { + componentStateGrid.invalidateRows(args.rows); + componentStateGrid.render(); + }); + + // hold onto an instance of the grid + $('#component-state-table').data('gridInstance', componentStateGrid); + + // initialize the number of display items + $('#displayed-component-state-entries').text('0'); + $('#total-component-state-entries').text('0'); + }, + + /** + * Shows the state for a given component. + * + * @param {object} component + * @param {boolean} canClear + */ + showState: function (component, canClear) { + return $.ajax({ + type: 'GET', + url: component.uri + '/state', + dataType: 'json' + }).done(function (response) { + var componentState = response.componentState; + var componentStateTable = $('#component-state-table'); + + // load the table + loadComponentState(componentState.localState, componentState.clusterState); + + // populate the name/description + $('#component-state-name').text(component.name); + $('#component-state-description').text(componentState.stateDescription).ellipsis(); + + // store the component + componentStateTable.data('component', component); + + // show the dialog + $('#component-state-dialog').modal('show'); + + // only activate the link when appropriate + if (canClear === false) { + $('#clear-link').addClass('disabled').attr('title', 'Component state can only be cleared when the component is not actively running'); + } + + // reset the grid size + var componentStateGrid = componentStateTable.data('gridInstance'); + componentStateGrid.resizeCanvas(); + }).fail(nf.Common.handleAjaxError); + } + }; +}()); \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-context-menu.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-context-menu.js index 089ac9ef60..f841e51186 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-context-menu.js +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-context-menu.js @@ -231,6 +231,30 @@ nf.ContextMenu = (function () { (nf.CanvasUtils.isInputPort(selection) && nf.Canvas.getParentGroupId() !== null); }; + /** + * Determines whether the current selection is a processor. + * + * @param {selection} selection + */ + var isStatefulProcessor = function (selection) { + // ensure the correct number of components are selected + if (selection.size() !== 1) { + return false; + } + + // ensure the user is DFM + if (nf.Common.isDFM() === false) { + return false; + } + + if (nf.CanvasUtils.isProcessor(selection)) { + var processorData = selection.datum(); + return processorData.component.persistsState === true; + } else { + return false; + } + }; + /** * Determines whether the current selection is a process group. * @@ -399,6 +423,7 @@ nf.ContextMenu = (function () { {condition: canStopTransmission, menuItem: {img: 'images/iconTransmissionInactive.png', text: 'Disable transmission', action: 'disableTransmission'}}, {condition: supportsStats, menuItem: {img: 'images/iconChart.png', text: 'Stats', action: 'showStats'}}, {condition: canAccessProvenance, menuItem: {img: 'images/iconProvenance.png', imgStyle: 'context-menu-provenance', text: 'Data provenance', action: 'openProvenance'}}, + {condition: isStatefulProcessor, menuItem: {img: 'images/iconViewState.png', text: 'View state', action: 'viewState'}}, {condition: canMoveToFront, menuItem: {img: 'images/iconToFront.png', text: 'Bring to front', action: 'toFront'}}, {condition: isConnection, menuItem: {img: 'images/iconGoTo.png', text: 'Go to source', action: 'showSource'}}, {condition: isConnection, menuItem: {img: 'images/iconGoTo.png', text: 'Go to destination', action: 'showDestination'}}, diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-processor-configuration.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-processor-configuration.js index 098f840b37..872555c037 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-processor-configuration.js +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-processor-configuration.js @@ -572,24 +572,33 @@ nf.ProcessorConfiguration = (function () { // get the processor details var processor = selectionData.component; + var requests = []; + // reload the processor in case an property descriptors have updated - var reloadProcessor = nf.Processor.reload(processor); - + requests.push(nf.Processor.reload(processor)); + // get the processor history - var loadHistory = $.ajax({ + requests.push($.ajax({ type: 'GET', url: '../nifi-api/controller/history/processors/' + encodeURIComponent(processor.id), dataType: 'json' - }); + })); + + // get the processor state if we're a DFM + if (nf.Common.isDFM()) { + requests.push(); + } // once everything is loaded, show the dialog - $.when(reloadProcessor, loadHistory).done(function (processorResponse, historyResponse) { + $.when.apply(window, requests).done(function (processorResponse, historyResponse, stateResponse) { // get the updated processor processor = processorResponse[0].processor; // get the processor history var processorHistory = historyResponse[0].componentHistory; - + + console.log(stateResponse); + // record the processor details $('#processor-configuration').data('processorDetails', processor); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-settings.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-settings.js index c07d52c8da..dcec5c0c5b 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-settings.js +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-settings.js @@ -171,6 +171,12 @@ nf.Settings = (function () { * Hides the selected controller service. */ var clearSelectedControllerService = function () { + if (nf.Canvas.isClustered()) { + $('#controller-service-availability-combo').combo('setSelectedOption', { + value: config.node + }); + } + $('#controller-service-type-description').text(''); $('#controller-service-type-name').text(''); $('#selected-controller-service-name').text(''); @@ -192,7 +198,7 @@ nf.Settings = (function () { /** * Performs the filtering. - * + * * @param {object} item The item subject to filtering * @param {object} args Filter arguments * @returns {Boolean} Whether or not to include the item @@ -224,7 +230,7 @@ nf.Settings = (function () { /** * Determines if the item matches the filter. - * + * * @param {object} item The item to filter * @param {object} args The filter criteria * @returns {boolean} Whether the item matches the filter @@ -250,7 +256,7 @@ nf.Settings = (function () { /** * Determines if the specified tags match all the tags selected by the user. - * + * * @argument {string[]} tagFilters The tag filters * @argument {string} tags The tags to test */ @@ -293,7 +299,7 @@ nf.Settings = (function () { /** * Adds a new controller service of the specified type. - * + * * @param {string} controllerServiceType */ var addControllerService = function (controllerServiceType) { @@ -478,7 +484,7 @@ nf.Settings = (function () { // end the udpate controllerServiceTypesData.endUpdate(); - + // set the total number of processors $('#total-controller-service-types, #displayed-controller-service-types').text(response.controllerServiceTypes.length); @@ -544,7 +550,7 @@ nf.Settings = (function () { /** * Formatter for the type column. - * + * * @param {type} row * @param {type} cell * @param {type} value @@ -558,7 +564,7 @@ nf.Settings = (function () { /** * Formatter for the availability column. - * + * * @param {type} row * @param {type} cell * @param {type} value @@ -576,7 +582,7 @@ nf.Settings = (function () { /** * Sorts the specified data using the specified sort details. - * + * * @param {object} sortDetails * @param {object} data */ @@ -628,24 +634,28 @@ nf.Settings = (function () { // more details formatter var moreControllerServiceDetails = function (row, cell, value, columnDef, dataContext) { var markup = ''; + + // always include a button to view the usage + markup += ''; + var hasErrors = !nf.Common.isEmpty(dataContext.validationErrors); var hasBulletins = !nf.Common.isEmpty(dataContext.bulletins); - + if (hasErrors) { - markup += ''; + markup += ''; } - + if (hasBulletins) { markup += ''; } - + if (hasErrors || hasBulletins) { markup += ''; } - + return markup; }; - + var controllerServiceStateFormatter = function (row, cell, value, columnDef, dataContext) { // determine the appropriate label var icon = '', label = ''; @@ -667,7 +677,7 @@ nf.Settings = (function () { label = 'Enabling'; } } - + // format the markup var formattedValue = '
'; return formattedValue + '
' + label + '
'; @@ -676,19 +686,23 @@ nf.Settings = (function () { var controllerServiceActionFormatter = function (row, cell, value, columnDef, dataContext) { var markup = ''; - // only DFMs can edit a controller service + // only DFMs can edit a controller service and view state if (nf.Common.isDFM()) { if (dataContext.state === 'ENABLED' || dataContext.state === 'ENABLING') { - markup += ' '; + markup += ''; } else if (dataContext.state === 'DISABLED') { - markup += ' '; - + markup += ''; + // if there are no validation errors allow enabling if (nf.Common.isEmpty(dataContext.validationErrors)) { - markup += ' '; + markup += ''; } - - markup += ' '; + + markup += ''; + } + + if (dataContext.persistsState === true) { + markup += ''; } } @@ -710,7 +724,7 @@ nf.Settings = (function () { if (nf.Canvas.isClustered()) { controllerServicesColumns.push({id: 'availability', field: 'availability', name: 'Availability', formatter: availabilityFormatter, sortable: true, resizeable: true}); } - + // action column should always be last controllerServicesColumns.push({id: 'actions', name: ' ', resizable: false, formatter: controllerServiceActionFormatter, sortable: false, width: 90, maxWidth: 90}); @@ -755,14 +769,20 @@ nf.Settings = (function () { nf.ControllerService.disable(controllerService); } else if (target.hasClass('delete-controller-service')) { nf.ControllerService.remove(controllerService); + } else if (target.hasClass('view-state-controller-service')) { + nf.ComponentState.showState(controllerService, controllerService.state === 'DISABLED'); + } + } else if (controllerServicesGrid.getColumns()[args.cell].id === 'moreDetails') { + if (target.hasClass('view-controller-service')) { + nf.ControllerService.showDetails(controllerService); } else if (target.hasClass('controller-service-usage')) { // close the settings dialog $('#shell-close-button').click(); - + // open the documentation for this reporting task nf.Shell.showPage('../nifi-docs/documentation?' + $.param({ - select: nf.Common.substringAfterLast(controllerService.type, '.') - })).done(function() { + select: nf.Common.substringAfterLast(controllerService.type, '.') + })).done(function() { nf.Settings.showSettings(); }); } @@ -812,14 +832,14 @@ nf.Settings = (function () { }, nf.Common.config.tooltipConfig)); } } - + var bulletinIcon = $(this).find('img.has-bulletins'); if (bulletinIcon.length && !bulletinIcon.data('qtip')) { var taskId = $(this).find('span.row-id').text(); // get the task item var item = controllerServicesData.getItemById(taskId); - + // format the tooltip var bulletins = nf.Common.getFormattedBulletins(item.bulletins); var tooltip = nf.Common.formatUnorderedList(bulletins); @@ -948,6 +968,12 @@ nf.Settings = (function () { * Hides the selected reporting task. */ var clearSelectedReportingTask = function () { + if (nf.Canvas.isClustered()) { + $('#reporting-task-availability-combo').combo('setSelectedOption', { + value: config.node + }); + } + $('#reporting-task-type-description').text(''); $('#reporting-task-type-name').text(''); $('#selected-reporting-task-name').text(''); @@ -969,7 +995,7 @@ nf.Settings = (function () { /** * Performs the filtering. - * + * * @param {object} item The item subject to filtering * @param {object} args Filter arguments * @returns {Boolean} Whether or not to include the item @@ -1015,10 +1041,10 @@ nf.Settings = (function () { addReportingTask(selectedTaskType); } }; - + /** * Adds a new reporting task of the specified type. - * + * * @param {string} reportingTaskType */ var addReportingTask = function (reportingTaskType) { @@ -1276,24 +1302,28 @@ nf.Settings = (function () { var moreReportingTaskDetails = function (row, cell, value, columnDef, dataContext) { var markup = ''; + + // always include a button to view the usage + markup += ''; + var hasErrors = !nf.Common.isEmpty(dataContext.validationErrors); var hasBulletins = !nf.Common.isEmpty(dataContext.bulletins); - + if (hasErrors) { - markup += ''; + markup += ''; } - + if (hasBulletins) { markup += ''; } - + if (hasErrors || hasBulletins) { markup += ''; } - + return markup; }; - + var reportingTaskRunStatusFormatter = function (row, cell, value, columnDef, dataContext) { // determine the appropriate label var label; @@ -1308,34 +1338,38 @@ nf.Settings = (function () { label = 'Disabled'; } } - + // include the active thread count if appropriate var activeThreadCount = ''; if (nf.Common.isDefinedAndNotNull(dataContext.activeThreadCount) && dataContext.activeThreadCount > 0) { activeThreadCount = '(' + dataContext.activeThreadCount + ')'; } - + // format the markup var formattedValue = '
'; return formattedValue + '
' + nf.Common.escapeHtml(label) + '
' + nf.Common.escapeHtml(activeThreadCount) + '
'; }; - + var reportingTaskActionFormatter = function (row, cell, value, columnDef, dataContext) { var markup = ''; - // only DFMs can edit reporting tasks + // only DFMs can edit reporting tasks and view state if (nf.Common.isDFM()) { if (dataContext.state === 'RUNNING') { - markup += ' '; + markup += ''; } else if (dataContext.state === 'STOPPED' || dataContext.state === 'DISABLED') { - markup += ' '; - + markup += ''; + // support starting when stopped and no validation errors if (dataContext.state === 'STOPPED' && nf.Common.isEmpty(dataContext.validationErrors)) { - markup += ' '; + markup += ''; } - - markup += ' '; + + markup += ''; + } + + if (dataContext.persistsState === true) { + markup += ''; } } @@ -1357,7 +1391,7 @@ nf.Settings = (function () { if (nf.Canvas.isClustered()) { reportingTasksColumnModel.push({id: 'availability', field: 'availability', name: 'Availability', formatter: availabilityFormatter, sortable: true, resizeable: true}); } - + // action column should always be last reportingTasksColumnModel.push({id: 'actions', name: ' ', resizable: false, formatter: reportingTaskActionFormatter, sortable: false, width: 90, maxWidth: 90}); @@ -1402,14 +1436,21 @@ nf.Settings = (function () { nf.ReportingTask.stop(reportingTask); } else if (target.hasClass('delete-reporting-task')) { nf.ReportingTask.remove(reportingTask); + } else if (target.hasClass('view-state-reporting-task')) { + var canClear = reportingTask.state === 'STOPPED' && reportingTask.activeThreadCount === 0; + nf.ComponentState.showState(reportingTask, canClear); + } + } else if (reportingTasksGrid.getColumns()[args.cell].id === 'moreDetails') { + if (target.hasClass('view-reporting-task')) { + nf.ReportingTask.showDetails(reportingTask); } else if (target.hasClass('reporting-task-usage')) { // close the settings dialog $('#shell-close-button').click(); - + // open the documentation for this reporting task nf.Shell.showPage('../nifi-docs/documentation?' + $.param({ - select: nf.Common.substringAfterLast(reportingTask.type, '.') - })).done(function() { + select: nf.Common.substringAfterLast(reportingTask.type, '.') + })).done(function() { nf.Settings.showSettings(); }); } @@ -1459,14 +1500,14 @@ nf.Settings = (function () { }, nf.Common.config.tooltipConfig)); } } - + var bulletinIcon = $(this).find('img.has-bulletins'); if (bulletinIcon.length && !bulletinIcon.data('qtip')) { var taskId = $(this).find('span.row-id').text(); // get the task item var item = reportingTasksData.getItemById(taskId); - + // format the tooltip var bulletins = nf.Common.getFormattedBulletins(item.bulletins); var tooltip = nf.Common.formatUnorderedList(bulletins); @@ -1611,26 +1652,26 @@ nf.Settings = (function () { var selectedTab = $('li.settings-selected-tab').text(); if (selectedTab === 'Controller Services') { $('#new-controller-service-dialog').modal('show'); - + // reset the canvas size after the dialog is shown var controllerServiceTypesGrid = $('#controller-service-types-table').data('gridInstance'); if (nf.Common.isDefinedAndNotNull(controllerServiceTypesGrid)) { controllerServiceTypesGrid.setSelectedRows([0]); controllerServiceTypesGrid.resizeCanvas(); } - + // set the initial focus $('#controller-service-type-filter').focus(); } else if (selectedTab === 'Reporting Tasks') { $('#new-reporting-task-dialog').modal('show'); - + // reset the canvas size after the dialog is shown var reportingTaskTypesGrid = $('#reporting-task-types-table').data('gridInstance'); if (nf.Common.isDefinedAndNotNull(reportingTaskTypesGrid)) { reportingTaskTypesGrid.setSelectedRows([0]); reportingTaskTypesGrid.resizeCanvas(); } - + // set the initial focus $('#reporting-task-type-filter').focus(); } @@ -1641,7 +1682,7 @@ nf.Settings = (function () { initControllerServices(); initReportingTasks(); }, - + /** * Update the size of the grid based on its container's current size. */ @@ -1656,7 +1697,7 @@ nf.Settings = (function () { reportingTasksGrid.resizeCanvas(); } }, - + /** * Shows the settings dialog. */ @@ -1666,11 +1707,11 @@ nf.Settings = (function () { // reset button state $('#settings-save').mouseout(); }); - + // adjust the table size nf.Settings.resetTableSize(); }, - + /** * Loads the settings. */ @@ -1715,10 +1756,10 @@ nf.Settings = (function () { } }).fail(nf.Common.handleAjaxError); }, - + /** * Sets the controller service and reporting task bulletins in their respective tables. - * + * * @param {object} controllerServiceBulletins * @param {object} reportingTaskBulletins */ @@ -1733,7 +1774,7 @@ nf.Settings = (function () { var controllerServiceBulletinsBySource = d3.nest() .key(function(d) { return d.sourceId; }) .map(controllerServiceBulletins, d3.map); - + controllerServiceBulletinsBySource.forEach(function(sourceId, sourceBulletins) { var controllerService = controllerServicesData.getItemById(sourceId); if (nf.Common.isDefinedAndNotNull(controllerService)) { @@ -1757,7 +1798,7 @@ nf.Settings = (function () { var reportingTasksGrid = $('#reporting-tasks-table').data('gridInstance'); var reportingTasksData = reportingTasksGrid.getData(); reportingTasksData.beginUpdate(); - + // if there are some bulletins process them if (!nf.Common.isEmpty(reportingTaskBulletins)) { var reportingTaskBulletinsBySource = d3.nest() diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/ListHDFS.java b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/ListHDFS.java index ea2d39776e..d624e6f591 100644 --- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/ListHDFS.java +++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/ListHDFS.java @@ -17,18 +17,13 @@ package org.apache.nifi.processors.hadoop; import java.io.File; -import java.io.FileInputStream; -import java.io.FileOutputStream; import java.io.IOException; import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; import java.util.Date; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Properties; import java.util.Set; import org.apache.hadoop.fs.FileStatus; @@ -38,6 +33,7 @@ import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.nifi.annotation.behavior.InputRequirement; import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.behavior.Stateful; import org.apache.nifi.annotation.behavior.TriggerSerially; import org.apache.nifi.annotation.behavior.TriggerWhenEmpty; import org.apache.nifi.annotation.behavior.WritesAttribute; @@ -45,9 +41,13 @@ import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; +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.DistributedMapCacheClient; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; @@ -59,7 +59,6 @@ import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.processors.hadoop.util.HDFSListing; import org.apache.nifi.processors.hadoop.util.StringSerDe; -import org.codehaus.jackson.JsonGenerationException; import org.codehaus.jackson.JsonNode; import org.codehaus.jackson.JsonParseException; import org.codehaus.jackson.map.JsonMappingException; @@ -87,13 +86,17 @@ import org.codehaus.jackson.map.ObjectMapper; @WritesAttribute(attribute="hdfs.permissions", description="The permissions for the file in HDFS. This is formatted as 3 characters for the owner, " + "3 for the group, and 3 for other users. For example rw-rw-r--") }) +@Stateful(scopes = Scope.CLUSTER, description = "After performing a listing of HDFS files, the timestamp of the newest file is stored, " + + "along with the filenames of all files that share that same timestamp. This allows the Processor to list only files that have been added or modified after " + + "this date the next time that the Processor is run. State is stored across the cluster so that this Processor can be run on Primary Node only and if a new Primary " + + "Node is selected, the new node can pick up where the previous node left off, without duplicating the data.") @SeeAlso({GetHDFS.class, FetchHDFS.class, PutHDFS.class}) public class ListHDFS extends AbstractHadoopProcessor { public static final PropertyDescriptor DISTRIBUTED_CACHE_SERVICE = new PropertyDescriptor.Builder() .name("Distributed Cache Service") .description("Specifies the Controller Service that should be used to maintain state about what has been pulled from HDFS so that if a new node " + "begins pulling data, it won't duplicate all of the work that has been done.") - .required(true) + .required(false) .identifiesControllerService(DistributedMapCacheClient.class) .build(); @@ -164,7 +167,7 @@ public class ListHDFS extends AbstractHadoopProcessor { @Override public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) { - if ( descriptor.equals(DIRECTORY) ) { + if (isConfigurationRestored() && descriptor.equals(DIRECTORY)) { lastListingTime = null; // clear lastListingTime so that we have to fetch new time latestPathsListed = new HashSet<>(); } @@ -176,70 +179,57 @@ public class ListHDFS extends AbstractHadoopProcessor { return mapper.readValue(jsonNode, HDFSListing.class); } + @OnScheduled + public void moveStateToStateManager(final ProcessContext context) throws IOException { + final StateManager stateManager = context.getStateManager(); + final StateMap stateMap = stateManager.getState(Scope.CLUSTER); - private Long getMinTimestamp(final String directory, final DistributedMapCacheClient client) throws IOException { - // Determine the timestamp for the last file that we've listed. + // Check if we have already stored state in the cluster state manager. + if (stateMap.getVersion() == -1L) { + final HDFSListing serviceListing = getListingFromService(context); + if (serviceListing != null) { + context.getStateManager().setState(serviceListing.toMap(), Scope.CLUSTER); + } + } + } + + private HDFSListing getListingFromService(final ProcessContext context) throws IOException { + final DistributedMapCacheClient client = context.getProperty(DISTRIBUTED_CACHE_SERVICE).asControllerService(DistributedMapCacheClient.class); + if (client == null) { + return null; + } + + final String directory = context.getProperty(DIRECTORY).getValue(); + final String remoteValue = client.get(getKey(directory), new StringSerDe(), new StringSerDe()); + if (remoteValue == null) { + return null; + } + + try { + return deserialize(remoteValue); + } catch (final Exception e) { + getLogger().error("Failed to retrieve state from Distributed Map Cache because the content that was retrieved could not be understood", e); + return null; + } + } + + + private Long getMinTimestamp(final String directory, final HDFSListing remoteListing) throws IOException { + // No cluster-wide state has been recovered. Just use whatever values we already have. + if (remoteListing == null) { + return lastListingTime; + } + + // If our local timestamp is already later than the remote listing's timestamp, use our local info. 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. - try { - final StringSerDe serde = new StringSerDe(); - final String serializedState = client.get(getKey(directory), serde, serde); - if ( serializedState == null || serializedState.isEmpty() ) { - minTimestamp = null; - this.latestPathsListed = Collections.emptySet(); - } else { - final HDFSListing listing = deserialize(serializedState); - this.lastListingTime = listing.getLatestTimestamp().getTime(); - minTimestamp = listing.getLatestTimestamp().getTime(); - this.latestPathsListed = listing.toPaths(); - } + if (minTimestamp != null && minTimestamp > remoteListing.getLatestTimestamp().getTime()) { + return minTimestamp; + } - 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 HDFSListing 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 (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 HDFS 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); - } + // Use the remote listing's information. + if (minTimestamp == null || electedPrimaryNode) { + this.latestPathsListed = remoteListing.toPaths(); + this.lastListingTime = remoteListing.getLatestTimestamp().getTime(); } return minTimestamp; @@ -248,11 +238,20 @@ public class ListHDFS extends AbstractHadoopProcessor { @Override public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { final String directory = context.getProperty(DIRECTORY).getValue(); - final DistributedMapCacheClient client = context.getProperty(DISTRIBUTED_CACHE_SERVICE).asControllerService(DistributedMapCacheClient.class); + // Ensure that we are using the latest listing information before we try to perform a listing of HDFS files. final Long minTimestamp; try { - minTimestamp = getMinTimestamp(directory, client); + final HDFSListing stateListing; + final StateMap stateMap = context.getStateManager().getState(Scope.CLUSTER); + if (stateMap.getVersion() == -1L) { + stateListing = null; + } else { + final Map stateValues = stateMap.toMap(); + stateListing = HDFSListing.fromMap(stateValues); + } + + minTimestamp = getMinTimestamp(directory, stateListing); } 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(); @@ -311,32 +310,19 @@ public class ListHDFS extends AbstractHadoopProcessor { // However, we want to save the state both locally and remotely. // We store the state remotely so that if a new Primary Node is chosen, it can pick up where the // 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 HDFSListing latestListing = createListing(latestListingModTime, statuses); + try { - serializedState = serializeState(latestListingModTime, statuses); - } catch (final Exception e) { - getLogger().error("Failed to serialize state due to {}", new Object[] {e}); - } - - if ( serializedState != null ) { - // Save our state locally. - try { - persistLocalState(directory, 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. - try { - client.put(getKey(directory), 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); - } + context.getStateManager().setState(latestListing.toMap(), Scope.CLUSTER); + } catch (final IOException ioe) { + getLogger().warn("Failed to save cluster-wide state. If NiFi is restarted, data duplication may occur", ioe); } lastListingTime = latestListingModTime; + latestPathsListed.clear(); + for (final FileStatus status : statuses) { + latestPathsListed.add(status.getPath()); + } } else { getLogger().debug("There is no data to list. Yielding."); context.yield(); @@ -372,71 +358,18 @@ public class ListHDFS extends AbstractHadoopProcessor { return statusSet; } - - private String serializeState(final long latestListingTime, final Set statuses) 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 ( statuses.isEmpty() ) { - return null; - } else { - final List sortedStatuses = new ArrayList<>(statuses); - Collections.sort(sortedStatuses, new Comparator() { - @Override - public int compare(final FileStatus o1, final FileStatus o2) { - return Long.compare(o1.getModificationTime(), o2.getModificationTime()); - } - }); - - final long latestListingModTime = sortedStatuses.get(sortedStatuses.size() - 1).getModificationTime(); - final Set pathsWithModTimeEqualToListingModTime = new HashSet<>(); - for (int i=sortedStatuses.size() - 1; i >= 0; i--) { - final FileStatus status = sortedStatuses.get(i); - if (status.getModificationTime() == latestListingModTime) { - pathsWithModTimeEqualToListingModTime.add(status.getPath()); - } - } - - this.latestPathsListed = pathsWithModTimeEqualToListingModTime; - - final HDFSListing listing = new HDFSListing(); - listing.setLatestTimestamp(new Date(latestListingModTime)); - final Set paths = new HashSet<>(); - for ( final Path path : pathsWithModTimeEqualToListingModTime ) { - paths.add(path.toUri().toString()); - } - listing.setMatchingPaths(paths); - - final ObjectMapper mapper = new ObjectMapper(); - final String serializedState = mapper.writerWithType(HDFSListing.class).writeValueAsString(listing); - return serializedState; - } - } - - protected void persistLocalState(final String directory, 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"); + private HDFSListing createListing(final long latestListingModTime, final Set statuses) { + final Set paths = new HashSet<>(); + for (final FileStatus status : statuses) { + final String path = status.getPath().toUri().toString(); + paths.add(path); } - final Properties props = new Properties(); - if ( persistenceFile.exists() ) { - try (final FileInputStream fis = new FileInputStream(persistenceFile)) { - props.load(fis); - } - } + final HDFSListing listing = new HDFSListing(); + listing.setLatestTimestamp(new Date(latestListingModTime)); + listing.setMatchingPaths(paths); - props.setProperty(directory, serializedState); - - try (final FileOutputStream fos = new FileOutputStream(persistenceFile)) { - props.store(fos, null); - } + return listing; } private String getAbsolutePath(final Path path) { diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/util/HDFSListing.java b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/util/HDFSListing.java index a4d957a1a7..6786945d2b 100644 --- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/util/HDFSListing.java +++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/util/HDFSListing.java @@ -16,9 +16,12 @@ */ package org.apache.nifi.processors.hadoop.util; +import java.util.ArrayList; import java.util.Collection; import java.util.Date; +import java.util.HashMap; import java.util.HashSet; +import java.util.Map; import java.util.Set; import javax.xml.bind.annotation.XmlTransient; @@ -35,6 +38,11 @@ public class HDFSListing { private Date latestTimestamp; private Collection matchingPaths; + public static class StateKeys { + public static final String TIMESTAMP = "timestamp"; + public static final String PATH_PREFIX = "path."; + } + /** * @return the modification date of the newest file that was contained in the HDFS Listing */ @@ -80,4 +88,41 @@ public class HDFSListing { this.matchingPaths = matchingPaths; } + /** + * Converts this HDFSListing into a Map<String, String> so that it can be stored in a StateManager. + * + * @return a Map that represents the same information as this HDFSListing + */ + public Map toMap() { + final Map map = new HashMap<>(1 + matchingPaths.size()); + map.put(StateKeys.TIMESTAMP, String.valueOf(latestTimestamp.getTime())); + + int counter = 0; + for (final String path : matchingPaths) { + map.put(StateKeys.PATH_PREFIX + String.valueOf(counter++), path); + } + + return map; + } + + public static HDFSListing fromMap(final Map map) { + if (map == null || map.isEmpty()) { + return null; + } + + final String timestampValue = map.get(StateKeys.TIMESTAMP); + final long timestamp = Long.parseLong(timestampValue); + + final Collection matchingPaths = new ArrayList<>(map.size() - 1); + for (final Map.Entry entry : map.entrySet()) { + if (entry.getKey().startsWith(StateKeys.PATH_PREFIX)) { + matchingPaths.add(entry.getValue()); + } + } + + final HDFSListing listing = new HDFSListing(); + listing.setLatestTimestamp(new Date(timestamp)); + listing.setMatchingPaths(matchingPaths); + return listing; + } } diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/TestListHDFS.java b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/TestListHDFS.java index c301bf3022..add89e8d68 100644 --- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/TestListHDFS.java +++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/TestListHDFS.java @@ -16,7 +16,7 @@ */ package org.apache.nifi.processors.hadoop; -import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertTrue; @@ -41,6 +41,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.util.Progressable; import org.apache.nifi.annotation.notification.PrimaryNodeState; +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; @@ -131,8 +132,8 @@ public class TestListHDFS { @Test - public void testNoListUntilUpdateFromRemoteOnPrimaryNodeChange() { - proc.fileSystem.addFileStatus(new Path("/test"), new FileStatus(1L, false, 1, 1L, 0L, 0L, create777(), "owner", "group", new Path("/test/testFile.txt"))); + public void testNoListUntilUpdateFromRemoteOnPrimaryNodeChange() throws IOException { + proc.fileSystem.addFileStatus(new Path("/test"), new FileStatus(1L, false, 1, 1L, 1999L, 0L, create777(), "owner", "group", new Path("/test/testFile.txt"))); runner.run(); @@ -145,7 +146,7 @@ public class TestListHDFS { runner.clearTransferState(); // add new file to pull - proc.fileSystem.addFileStatus(new Path("/test"), new FileStatus(1L, false, 1, 1L, 0L, 0L, create777(), "owner", "group", new Path("/test/testFile2.txt"))); + proc.fileSystem.addFileStatus(new Path("/test"), new FileStatus(1L, false, 1, 1L, 1999L, 0L, create777(), "owner", "group", new Path("/test/testFile2.txt"))); // trigger primary node change proc.onPrimaryNodeChange(PrimaryNodeState.ELECTED_PRIMARY_NODE); @@ -153,29 +154,45 @@ public class TestListHDFS { // cause calls to service to fail service.failOnCalls = true; - runner.run(); - runner.assertAllFlowFilesTransferred(ListHDFS.REL_SUCCESS, 0); + runner.getStateManager().setFailOnStateGet(Scope.CLUSTER, true); - runner.run(); - runner.assertAllFlowFilesTransferred(ListHDFS.REL_SUCCESS, 0); - - final String key = proc.getKey("/test"); - - // wait just to a bit to ensure that the timestamp changes when we update the service - final Object curVal = service.values.get(key); + // Should fail to perform @OnScheduled methods. try { - Thread.sleep(10L); - } catch (final InterruptedException ie) { + runner.run(); + Assert.fail("Processor ran successfully"); + } catch (final AssertionError e) { } + runner.assertAllFlowFilesTransferred(ListHDFS.REL_SUCCESS, 0); + + // Should fail to perform @OnScheduled methods. + try { + runner.run(); + Assert.fail("Processor ran successfully"); + } catch (final AssertionError e) { + } + + runner.assertAllFlowFilesTransferred(ListHDFS.REL_SUCCESS, 0); + service.failOnCalls = false; + runner.getStateManager().setFailOnStateGet(Scope.CLUSTER, false); + runner.run(); runner.assertAllFlowFilesTransferred(ListHDFS.REL_SUCCESS, 1); - // ensure state saved both locally & remotely - assertTrue(proc.localStateSaved); - assertNotNull(service.values.get(key)); - assertNotSame(curVal, service.values.get(key)); + // ensure state saved + runner.getStateManager().assertStateSet(Scope.CLUSTER); + final Map newState = runner.getStateManager().getState(Scope.CLUSTER).toMap(); + assertEquals(3, newState.size()); + + final String path0 = newState.get("path.0"); + final String path1 = newState.get("path.1"); + assertTrue(path0.equals("/test/testFile.txt") || path0.equals("/test/testFile2.txt")); + assertTrue(path1.equals("/test/testFile.txt") || path1.equals("/test/testFile2.txt")); + assertNotSame(path0, path1); + + final Long timestamp = Long.parseLong(newState.get("timestamp")); + assertEquals(1999L, timestamp.longValue()); } @@ -186,7 +203,6 @@ public class TestListHDFS { private class ListHDFSWithMockedFileSystem extends ListHDFS { private final MockFileSystem fileSystem = new MockFileSystem(); - private boolean localStateSaved = false; @Override protected FileSystem getFileSystem() { @@ -202,12 +218,6 @@ public class TestListHDFS { protected FileSystem getFileSystem(final Configuration config) throws IOException { return fileSystem; } - - @Override - protected void persistLocalState(final String directory, final String serializedState) throws IOException { - super.persistLocalState(directory, serializedState); - localStateSaved = true; - } } diff --git a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/GetHBase.java b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/GetHBase.java index 98a612cd31..fa4d80a17f 100644 --- a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/GetHBase.java +++ b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/GetHBase.java @@ -16,8 +16,30 @@ */ package org.apache.nifi.hbase; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.ObjectInputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + import org.apache.commons.lang3.StringUtils; import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.Stateful; import org.apache.nifi.annotation.behavior.TriggerSerially; import org.apache.nifi.annotation.behavior.TriggerWhenEmpty; import org.apache.nifi.annotation.behavior.WritesAttribute; @@ -32,6 +54,9 @@ import org.apache.nifi.components.AllowableValue; 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.StateManager; +import org.apache.nifi.components.state.StateMap; import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.hbase.io.JsonRowSerializer; @@ -50,28 +75,6 @@ import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.util.ObjectHolder; -import java.io.File; -import java.io.FileInputStream; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.nio.charset.Charset; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.regex.Pattern; - @TriggerWhenEmpty @TriggerSerially @InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN) @@ -84,6 +87,9 @@ import java.util.regex.Pattern; @WritesAttribute(attribute = "hbase.table", description = "The name of the HBase table that the data was pulled from"), @WritesAttribute(attribute = "mime.type", description = "Set to application/json to indicate that output is JSON") }) +@Stateful(scopes = Scope.CLUSTER, description = "After performing a fetching from HBase, stores a timestamp of the last-modified cell that was found. In addition, it stores the ID of the row(s) " + + "and the value of each cell that has that timestamp as its modification date. This is stored across the cluster and allows the next fetch to avoid duplicating data, even if this Processor is " + + "run on Primary Node only and the Primary Node changes.") public class GetHBase extends AbstractProcessor { static final Pattern COLUMNS_PATTERN = Pattern.compile("\\w+(:\\w+)?(?:,\\w+(:\\w+)?)*"); @@ -101,7 +107,7 @@ public class GetHBase extends AbstractProcessor { .name("Distributed Cache Service") .description("Specifies the Controller Service that should be used to maintain state about what has been pulled from HBase" + " so that if a new node begins pulling data, it won't duplicate all of the work that has been done.") - .required(true) + .required(false) .identifiesControllerService(DistributedMapCacheClient.class) .build(); static final PropertyDescriptor CHARSET = new PropertyDescriptor.Builder() @@ -150,7 +156,7 @@ public class GetHBase extends AbstractProcessor { private volatile ScanResult lastResult = null; private volatile List columns = new ArrayList<>(); - private volatile boolean electedPrimaryNode = false; + private volatile boolean justElectedPrimaryNode = false; private volatile String previousTable = null; @Override @@ -197,7 +203,20 @@ public class GetHBase extends AbstractProcessor { } @OnScheduled - public void parseColumns(final ProcessContext context) { + public void parseColumns(final ProcessContext context) throws IOException { + final StateMap stateMap = context.getStateManager().getState(Scope.CLUSTER); + if (stateMap.getVersion() < 0) { + // no state has been stored in the State Manager - check if we have state stored in the + // DistributedMapCacheClient service and migrate it if so + final DistributedMapCacheClient client = context.getProperty(DISTRIBUTED_CACHE_SERVICE).asControllerService(DistributedMapCacheClient.class); + final ScanResult scanResult = getState(client); + if (scanResult != null) { + storeState(scanResult, context.getStateManager()); + } + + clearState(client); + } + final String columnsValue = context.getProperty(COLUMNS).getValue(); final String[] columns = (columnsValue == null || columnsValue.isEmpty() ? new String[0] : columnsValue.split(",")); @@ -217,15 +236,15 @@ public class GetHBase extends AbstractProcessor { @OnPrimaryNodeStateChange public void onPrimaryNodeChange(final PrimaryNodeState newState) { - if (newState == PrimaryNodeState.ELECTED_PRIMARY_NODE) { - electedPrimaryNode = true; - } + justElectedPrimaryNode = (newState == PrimaryNodeState.ELECTED_PRIMARY_NODE); } @OnRemoved public void onRemoved(final ProcessContext context) { final DistributedMapCacheClient client = context.getProperty(DISTRIBUTED_CACHE_SERVICE).asControllerService(DistributedMapCacheClient.class); - clearState(client); + if (client != null) { + clearState(client); + } } @Override @@ -234,11 +253,14 @@ public class GetHBase extends AbstractProcessor { final String initialTimeRange = context.getProperty(INITIAL_TIMERANGE).getValue(); final String filterExpression = context.getProperty(FILTER_EXPRESSION).getValue(); final HBaseClientService hBaseClientService = context.getProperty(HBASE_CLIENT_SERVICE).asControllerService(HBaseClientService.class); - final DistributedMapCacheClient client = context.getProperty(DISTRIBUTED_CACHE_SERVICE).asControllerService(DistributedMapCacheClient.class); // if the table was changed then remove any previous state if (previousTable != null && !tableName.equals(previousTable)) { - clearState(client); + try { + context.getStateManager().clear(Scope.CLUSTER); + } catch (final IOException ioe) { + getLogger().warn("Failed to clear Cluster State", ioe); + } previousTable = tableName; } @@ -246,7 +268,7 @@ public class GetHBase extends AbstractProcessor { final Charset charset = Charset.forName(context.getProperty(CHARSET).getValue()); final RowSerializer serializer = new JsonRowSerializer(charset); - this.lastResult = getState(client); + this.lastResult = getState(context.getStateManager()); final long defaultMinTime = (initialTimeRange.equals(NONE.getValue()) ? 0L : System.currentTimeMillis()); final long minTime = (lastResult == null ? defaultMinTime : lastResult.getTimestamp()); @@ -387,9 +409,8 @@ public class GetHBase extends AbstractProcessor { lastResult = scanResult; } - // save state to local storage and to distributed cache - persistState(client, lastResult); - + // save state using the framework's state manager + storeState(lastResult, context.getStateManager()); } catch (final IOException e) { getLogger().error("Failed to receive data from HBase due to {}", e); session.rollback(); @@ -421,27 +442,11 @@ public class GetHBase extends AbstractProcessor { return columns; } - private void persistState(final DistributedMapCacheClient client, final ScanResult scanResult) { - final File stateDir = getStateDir(); - if (!stateDir.exists()) { - stateDir.mkdirs(); - } - - final File file = getStateFile(); - try (final OutputStream fos = new FileOutputStream(file); - final ObjectOutputStream oos = new ObjectOutputStream(fos)) { - oos.writeObject(scanResult); - } 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); - } - - try { - client.put(getKey(), scanResult, new StringSerDe(), new ObjectSerDe()); - } catch (final IOException ioe) { - getLogger().warn("Unable to communicate with distributed cache server due to {}. Persisting state locally instead.", ioe); - } + private void storeState(final ScanResult scanResult, final StateManager stateManager) throws IOException { + stateManager.setState(scanResult.toFlatMap(), Scope.CLUSTER); } + private void clearState(final DistributedMapCacheClient client) { final File localState = getStateFile(); if (localState.exists()) { @@ -455,23 +460,35 @@ public class GetHBase extends AbstractProcessor { } } + + private ScanResult getState(final StateManager stateManager) throws IOException { + final StateMap stateMap = stateManager.getState(Scope.CLUSTER); + if (stateMap.getVersion() < 0) { + return null; + } + + return ScanResult.fromFlatMap(stateMap.toMap()); + } + private ScanResult getState(final DistributedMapCacheClient client) throws IOException { final StringSerDe stringSerDe = new StringSerDe(); final ObjectSerDe objectSerDe = new ObjectSerDe(); ScanResult scanResult = lastResult; // if we have no previous result, or we just became primary, pull from distributed cache - if (scanResult == null || electedPrimaryNode) { - final Object obj = client.get(getKey(), stringSerDe, objectSerDe); - if (obj == null || !(obj instanceof ScanResult)) { - scanResult = null; - } else { - scanResult = (ScanResult) obj; - getLogger().debug("Retrieved state from the distributed cache, previous timestamp was {}" , new Object[] {scanResult.getTimestamp()}); + if (scanResult == null || justElectedPrimaryNode) { + if (client != null) { + final Object obj = client.get(getKey(), stringSerDe, objectSerDe); + if (obj == null || !(obj instanceof ScanResult)) { + scanResult = null; + } else { + scanResult = (ScanResult) obj; + getLogger().debug("Retrieved state from the distributed cache, previous timestamp was {}", new Object[] {scanResult.getTimestamp()}); + } } // no requirement to pull an update from the distributed cache anymore. - electedPrimaryNode = false; + justElectedPrimaryNode = false; } // Check the persistence file. We want to use the latest timestamp that we have so that @@ -487,16 +504,6 @@ public class GetHBase extends AbstractProcessor { if (scanResult == null || localScanResult.getTimestamp() > scanResult.getTimestamp()) { scanResult = localScanResult; getLogger().debug("Using last timestamp from local state because it was newer than the distributed cache, or no value existed in the cache"); - - // Our local persistence file shows a later time than the Distributed service. - // Update the distributed service to match our local state. - try { - client.put(getKey(), localScanResult, stringSerDe, objectSerDe); - } catch (final IOException ioe) { - getLogger().warn("Local timestamp is {}, which is later than Distributed state but failed to update Distributed " - + "state due to {}. If a new node performs GetHBase Listing, data duplication may occur", - new Object[] {localScanResult.getTimestamp(), ioe}); - } } } } catch (final IOException | ClassNotFoundException ioe) { @@ -514,6 +521,13 @@ public class GetHBase extends AbstractProcessor { private final long latestTimestamp; private final Map> matchingCellHashes; + private static final Pattern CELL_ID_PATTERN = Pattern.compile(Pattern.quote(StateKeys.ROW_ID_PREFIX) + "(\\d+)(\\.(\\d+))?"); + + public static class StateKeys { + public static final String TIMESTAMP = "timestamp"; + public static final String ROW_ID_PREFIX = "row."; + } + public ScanResult(final long timestamp, final Map> cellHashes) { latestTimestamp = timestamp; matchingCellHashes = cellHashes; @@ -543,6 +557,81 @@ public class GetHBase extends AbstractProcessor { final String cellHash = new String(cellValue, StandardCharsets.UTF_8); return cellHashes.contains(cellHash); } + + public Map toFlatMap() { + final Map map = new HashMap<>(); + map.put(StateKeys.TIMESTAMP, String.valueOf(latestTimestamp)); + + int rowCounter = 0; + for (final Map.Entry> entry : matchingCellHashes.entrySet()) { + final String rowId = entry.getKey(); + + final String rowIdKey = StateKeys.ROW_ID_PREFIX + rowCounter; + final String cellKeyPrefix = rowIdKey + "."; + map.put(rowIdKey, rowId); + + final Set cellValues = entry.getValue(); + int cellCounter = 0; + for (final String cellValue : cellValues) { + final String cellId = cellKeyPrefix + (cellCounter++); + map.put(cellId, cellValue); + } + + rowCounter++; + } + + return map; + } + + public static ScanResult fromFlatMap(final Map map) { + if (map == null) { + return null; + } + + final String timestampValue = map.get(StateKeys.TIMESTAMP); + if (timestampValue == null) { + return null; + } + + final long timestamp = Long.parseLong(timestampValue); + final Map> rowIndexToMatchingCellHashes = new HashMap<>(); + final Map rowIndexToId = new HashMap<>(); + + for (final Map.Entry entry : map.entrySet()) { + final String key = entry.getKey(); + final Matcher matcher = CELL_ID_PATTERN.matcher(key); + if (!matcher.matches()) { + // if it's not a valid key, move on. + continue; + } + + final String rowIndex = matcher.group(1); + final String cellIndex = matcher.group(3); + + Set cellHashes = rowIndexToMatchingCellHashes.get(rowIndex); + if (cellHashes == null) { + cellHashes = new HashSet<>(); + rowIndexToMatchingCellHashes.put(rowIndex, cellHashes); + } + + if (cellIndex == null) { + // this provides a Row ID. + rowIndexToId.put(rowIndex, entry.getValue()); + } else { + cellHashes.add(entry.getValue()); + } + } + + final Map> matchingCellHashes = new HashMap<>(rowIndexToMatchingCellHashes.size()); + for (final Map.Entry> entry : rowIndexToMatchingCellHashes.entrySet()) { + final String rowIndex = entry.getKey(); + final String rowId = rowIndexToId.get(rowIndex); + final Set cellValues = entry.getValue(); + matchingCellHashes.put(rowId, cellValues); + } + + return new ScanResult(timestamp, matchingCellHashes); + } } } diff --git a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/test/java/org/apache/nifi/hbase/TestGetHBase.java b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/test/java/org/apache/nifi/hbase/TestGetHBase.java index 92f42f2e6a..8f6d8907a4 100644 --- a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/test/java/org/apache/nifi/hbase/TestGetHBase.java +++ b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/test/java/org/apache/nifi/hbase/TestGetHBase.java @@ -16,11 +16,31 @@ */ package org.apache.nifi.hbase; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotSame; +import static org.junit.Assert.assertTrue; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.CharacterCodingException; +import java.nio.charset.Charset; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + import org.apache.nifi.annotation.notification.PrimaryNodeState; +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; import org.apache.nifi.distributed.cache.client.Serializer; +import org.apache.nifi.hbase.GetHBase.ScanResult; import org.apache.nifi.hbase.scan.Column; import org.apache.nifi.hbase.util.StringSerDe; import org.apache.nifi.reporting.InitializationException; @@ -31,17 +51,6 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import java.io.File; -import java.io.IOException; -import java.nio.charset.CharacterCodingException; -import java.nio.charset.Charset; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; - public class TestGetHBase { private TestRunner runner; @@ -148,27 +157,17 @@ public class TestGetHBase { hBaseClient.addResult("row4", cells, now + 1); runner.run(); runner.assertAllFlowFilesTransferred(GetHBase.REL_SUCCESS, 5); + runner.clearTransferState(); proc = new MockGetHBase(stateFile); - final TestRunner newRunner = TestRunners.newTestRunner(proc); - - newRunner.addControllerService("cacheClient", cacheClient); - newRunner.enableControllerService(cacheClient); - - newRunner.addControllerService("hbaseClient", hBaseClient); - newRunner.enableControllerService(hBaseClient); - - newRunner.setProperty(GetHBase.TABLE_NAME, "nifi"); - newRunner.setProperty(GetHBase.DISTRIBUTED_CACHE_SERVICE, "cacheClient"); - newRunner.setProperty(GetHBase.HBASE_CLIENT_SERVICE, "hbaseClient"); hBaseClient.addResult("row0", cells, now - 2); hBaseClient.addResult("row1", cells, now - 1); hBaseClient.addResult("row2", cells, now - 1); hBaseClient.addResult("row3", cells, now); - newRunner.run(100); - newRunner.assertAllFlowFilesTransferred(GetHBase.REL_SUCCESS, 0); + runner.run(100); + runner.assertAllFlowFilesTransferred(GetHBase.REL_SUCCESS, 0); } @Test @@ -271,8 +270,7 @@ public class TestGetHBase { runner.assertAllFlowFilesTransferred(GetHBase.REL_SUCCESS, 4); // should have a local state file and a cache entry before removing - Assert.assertTrue(proc.getStateFile().exists()); - Assert.assertTrue(cacheClient.containsKey(proc.getKey(), new StringSerDe())); + runner.getStateManager().assertStateSet(Scope.CLUSTER); proc.onRemoved(runner.getProcessContext()); @@ -331,7 +329,7 @@ public class TestGetHBase { } @Test - public void testParseColumns() { + public void testParseColumns() throws IOException { runner.setProperty(GetHBase.COLUMNS, "cf1,cf2:cq1,cf3"); proc.parseColumns(runner.getProcessContext()); @@ -365,6 +363,47 @@ public class TestGetHBase { runner.assertNotValid(); } + + @Test + public void testScanResultConvert() { + final long timestamp = 14L; + final Map> cellHashes = new LinkedHashMap<>(); + + final Set row1Cells = new HashSet<>(); + row1Cells.add("hello"); + row1Cells.add("there"); + cellHashes.put("abc", row1Cells); + + final Set row2Cells = new HashSet<>(); + row2Cells.add("good-bye"); + row2Cells.add("there"); + cellHashes.put("xyz", row2Cells); + + final ScanResult scanResult = new GetHBase.ScanResult(timestamp, cellHashes); + + final Map flatMap = scanResult.toFlatMap(); + assertEquals(7, flatMap.size()); + assertEquals("abc", flatMap.get("row.0")); + + final String row0Cell0 = flatMap.get("row.0.0"); + final String row0Cell1 = flatMap.get("row.0.1"); + assertTrue(row0Cell0.equals("hello") || row0Cell0.equals("there")); + assertTrue(row0Cell1.equals("hello") || row0Cell1.equals("there")); + assertNotSame(row0Cell0, row0Cell1); + + assertEquals("xyz", flatMap.get("row.1")); + final String row1Cell0 = flatMap.get("row.1.0"); + final String row1Cell1 = flatMap.get("row.1.1"); + assertTrue(row1Cell0.equals("good-bye") || row1Cell0.equals("there")); + assertTrue(row1Cell1.equals("good-bye") || row1Cell1.equals("there")); + assertNotSame(row1Cell0, row1Cell1); + + final ScanResult reverted = ScanResult.fromFlatMap(flatMap); + assertEquals(timestamp, reverted.getTimestamp()); + assertEquals(cellHashes, reverted.getMatchingCells()); + } + + // Mock processor to override the location of the state file private static class MockGetHBase extends GetHBase { 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-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestInvokeGroovy.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestInvokeGroovy.java index 349b25d3eb..2dc700dd46 100644 --- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestInvokeGroovy.java +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestInvokeGroovy.java @@ -80,7 +80,8 @@ public class TestInvokeGroovy extends BaseScriptTest { context.setProperty(InvokeScriptedProcessor.SCRIPT_ENGINE, "Groovy"); context.setProperty(InvokeScriptedProcessor.SCRIPT_FILE, "target/test/resources/groovy/test_reader.groovy"); context.setProperty(InvokeScriptedProcessor.MODULES, "target/test/resources/groovy"); - processor.customValidate(new MockValidationContext(context)); + // State Manger is unused, and a null reference is specified + processor.customValidate(new MockValidationContext(context, null)); processor.setup(context); List descriptors = processor.getSupportedPropertyDescriptors(); @@ -112,7 +113,8 @@ public class TestInvokeGroovy extends BaseScriptTest { context.setProperty(InvokeScriptedProcessor.SCRIPT_ENGINE, "Groovy"); context.setProperty(InvokeScriptedProcessor.SCRIPT_FILE, "target/test/resources/groovy/test_reader.groovy"); - processor.customValidate(new MockValidationContext(context)); + // State Manger is unused, and a null reference is specified + processor.customValidate(new MockValidationContext(context, null)); processor.setup(context); Set relationships = processor.getRelationships(); 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..246f71af39 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,24 +19,27 @@ 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; import java.util.Properties; import java.util.Set; +import org.apache.nifi.annotation.behavior.Stateful; 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 +53,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; @@ -139,6 +141,8 @@ import org.codehaus.jackson.map.ObjectMapper; * */ @TriggerSerially +@Stateful(scopes={Scope.LOCAL, Scope.CLUSTER}, description="After a listing of resources is performed, the latest timestamp of any of the resources is stored in the component's state, " + + "along with all resources that have that same timestmap so that the Processor can avoid data duplication. The scope used depends on the implementation.") public abstract class AbstractListProcessor extends AbstractProcessor { public static final PropertyDescriptor DISTRIBUTED_CACHE_SERVICE = new PropertyDescriptor.Builder() .name("Distributed Cache Service") @@ -157,9 +161,13 @@ public abstract class AbstractListProcessor extends Ab .build(); - private volatile Long lastListingTime = null; private volatile Set latestIdentifiersListed = new HashSet<>(); - private volatile boolean electedPrimaryNode = false; + private volatile Long lastListingTime = null; + private volatile boolean justElectedPrimaryNode = 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()); @@ -174,9 +182,10 @@ public abstract class AbstractListProcessor extends Ab @Override public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) { - if (isListingResetNecessary(descriptor)) { + if (isConfigurationRestored() && isListingResetNecessary(descriptor)) { lastListingTime = null; // clear lastListingTime so that we have to fetch new time latestIdentifiersListed = new HashSet<>(); + resetListing = true; } } @@ -187,15 +196,115 @@ 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) { - electedPrimaryNode = true; + justElectedPrimaryNode = (newState == PrimaryNodeState.ELECTED_PRIMARY_NODE); + } + + @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(getStateScope(context)); + 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(), getStateScope(context)); + } 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(getStateScope(context)); + 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 + * @param scope the scope to use + * @throws IOException if unable to retrieve or store the state + */ + private void migrateState(final String path, final DistributedMapCacheClient client, final StateManager stateManager, final Scope scope) 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, scope); + } + } + + private void persist(final long timestamp, final Collection identifiers, final StateManager stateManager, final Scope scope) 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); + } + + protected String getKey(final String directory) { + return getIdentifier() + ".lastListingTime." + directory; } private EntityListing deserialize(final String serializedState) throws JsonParseException, JsonMappingException, IOException { @@ -205,154 +314,38 @@ 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 || justElectedPrimaryNode) { + final StateMap stateMap = context.getStateManager().getState(getStateScope(context)); + 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); + } + } + + justElectedPrimaryNode = false; + } } 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."); + getLogger().error("Failed to retrieve timestamp of last listing from the State Manager. Will not perform listing until this is accomplished."); context.yield(); return; } @@ -371,11 +364,12 @@ public abstract class AbstractListProcessor extends Ab return; } - int listCount = 0; Long latestListingTimestamp = null; + final List newEntries = new ArrayList<>(); for (final T entity : entityList) { - final boolean list = (minTimestamp == null || entity.getTimestamp() > minTimestamp - || (entity.getTimestamp() == minTimestamp && !latestIdentifiersListed.contains(entity.getIdentifier()))); + final boolean newTimestamp = minTimestamp == null || entity.getTimestamp() > minTimestamp; + final boolean newEntryForTimestamp = minTimestamp != null && entity.getTimestamp() == minTimestamp && !latestIdentifiersListed.contains(entity.getIdentifier()); + final boolean list = newTimestamp || newEntryForTimestamp; // Create the FlowFile for this path. if (list) { @@ -383,7 +377,14 @@ public abstract class AbstractListProcessor extends Ab FlowFile flowFile = session.create(); flowFile = session.putAllAttributes(flowFile, attributes); session.transfer(flowFile, REL_SUCCESS); - listCount++; + + // If we don't have a new timestamp but just have a new entry, we need to + // add all of the previous entries to our entityList. If we have a new timestamp, + // then the previous entries can go away. + if (!newTimestamp) { + newEntries.addAll(entityList); + } + newEntries.add(entity); if (latestListingTimestamp == null || entity.getTimestamp() > latestListingTimestamp) { latestListingTimestamp = entity.getTimestamp(); @@ -391,6 +392,7 @@ public abstract class AbstractListProcessor extends Ab } } + final int listCount = newEntries.size(); if (listCount > 0) { getLogger().info("Successfully created listing with {} new objects", new Object[] {listCount}); session.commit(); @@ -403,32 +405,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<>(newEntries.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 : newEntries) { + identifiers.add(entity.getIdentifier()); } + persist(latestListingTimestamp, identifiers, context.getStateManager(), getStateScope(context)); + } catch (final IOException ioe) { + getLogger().warn("Unable to save state due to {}. If NiFi is 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(); @@ -487,6 +476,13 @@ public abstract class AbstractListProcessor extends Ab */ protected abstract boolean isListingResetNecessary(final PropertyDescriptor property); + /** + * Returns a Scope that specifies where the state should be managed for this Processor + * + * @param context the ProcessContext to use in order to make a determination + * @return a Scope that specifies where the state should be managed for this Processor + */ + protected abstract Scope getStateScope(final ProcessContext context); private static class StringSerDe implements Serializer, Deserializer { 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 079f256b32..999bead25e 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; @@ -72,15 +65,18 @@ import org.apache.http.impl.conn.BasicHttpClientConnectionManager; import org.apache.http.ssl.SSLContextBuilder; import org.apache.nifi.annotation.behavior.InputRequirement; import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.behavior.Stateful; 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.StateManager; +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; @@ -96,16 +92,20 @@ import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.ssl.SSLContextService; import org.apache.nifi.ssl.SSLContextService.ClientAuth; import org.apache.nifi.util.StopWatch; +import org.apache.nifi.util.Tuple; @Tags({"get", "fetch", "poll", "http", "https", "ingest", "source", "input"}) @InputRequirement(Requirement.INPUT_FORBIDDEN) @CapabilityDescription("Fetches data from an HTTP or HTTPS URL and writes the data to the content of a FlowFile. Once the content has been fetched, the ETag and Last Modified " - + "dates are remembered (if the web server supports these concepts). This allows the Processor to fetch new data only if the remote data has changed. That is, once the " - + "content has been fetched from the given URL, it will not be fetched again until the content on the remote server changes.") + + "dates are remembered (if the web server supports these concepts). This allows the Processor to fetch new data only if the remote data has changed or until the state is cleared. That is, " + + "once the content has been fetched from the given URL, it will not be fetched again until the content on the remote server changes. Note that due to limitations on state " + + "management, stored \"last modified\" and etag fields never expire. If the URL in GetHttp uses Expression Language that is unbounded, there " + + "is the potential for Out of Memory Errors to occur.") @WritesAttributes({ @WritesAttribute(attribute = "filename", description = "The filename is set to the name of the file on the remote server"), @WritesAttribute(attribute = "mime.type", description = "The MIME Type of the FlowFile, as reported by the HTTP Content-Type header") }) +@Stateful(scopes = {Scope.LOCAL}, description = "Stores Last Modified Time and ETag headers returned by server so that the same data will not be fetched multiple times.") public class GetHTTP extends AbstractSessionFactoryProcessor { static final int PERSISTENCE_INTERVAL_MSEC = 10000; @@ -204,30 +204,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) { @@ -249,16 +233,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 @@ -273,28 +247,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); } } @@ -446,8 +405,22 @@ 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()); + final StateMap beforeStateMap; + + try { + beforeStateMap = context.getStateManager().getState(Scope.LOCAL); + final String lastModified = beforeStateMap.get(LAST_MODIFIED+":" + url); + if (lastModified != null) { + get.addHeader(HEADER_IF_MODIFIED_SINCE, parseStateValue(lastModified).getValue()); + } + + final String etag = beforeStateMap.get(ETAG+":" + url); + if (etag != null) { + get.addHeader(HEADER_IF_NONE_MATCH, parseStateValue(etag).getValue()); + } + } catch (final IOException ioe) { + throw new ProcessException(ioe); + } final String accept = context.getProperty(ACCEPT_CONTENT_TYPE).getValue(); if (accept != null) { @@ -494,42 +467,9 @@ 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 Header lastModified = response.getFirstHeader(HEADER_LAST_MODIFIED); - if (lastModified != null) { - lastModifiedRef.set(lastModified.getValue()); - } - final Header etag = response.getFirstHeader(HEADER_ETAG); - if (etag != null) { - entityTagRef.set(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(); - } - } + updateStateMap(context,response,beforeStateMap,url); + } catch (final IOException e) { context.yield(); session.rollback(); @@ -545,4 +485,70 @@ public class GetHTTP extends AbstractSessionFactoryProcessor { conMan.shutdown(); } } + + private void updateStateMap(ProcessContext context, HttpResponse response, StateMap beforeStateMap, String url){ + try { + Map workingMap = new HashMap<>(); + workingMap.putAll(beforeStateMap.toMap()); + final StateManager stateManager = context.getStateManager(); + StateMap oldValue = beforeStateMap; + + long currentTime = System.currentTimeMillis(); + + final Header receivedLastModified = response.getFirstHeader(HEADER_LAST_MODIFIED); + if (receivedLastModified != null) { + workingMap.put(LAST_MODIFIED + ":" + url, currentTime+":"+receivedLastModified.getValue()); + } + + final Header receivedEtag = response.getFirstHeader(HEADER_ETAG); + if (receivedEtag != null) { + workingMap.put(ETAG + ":" + url, currentTime+":"+receivedEtag.getValue()); + } + + boolean replaceSucceeded = stateManager.replace(oldValue, workingMap, Scope.LOCAL); + boolean changed; + + while(!replaceSucceeded){ + oldValue = stateManager.getState(Scope.LOCAL); + workingMap.clear(); + workingMap.putAll(oldValue.toMap()); + + changed = false; + + if(receivedLastModified != null){ + Tuple storedLastModifiedTuple = parseStateValue(workingMap.get(LAST_MODIFIED+":"+url)); + + if(Long.parseLong(storedLastModifiedTuple.getKey()) < currentTime){ + workingMap.put(LAST_MODIFIED + ":" + url, currentTime+":"+receivedLastModified.getValue()); + changed = true; + } + } + + if(receivedEtag != null){ + Tuple storedLastModifiedTuple = parseStateValue(workingMap.get(ETAG+":"+url)); + + if(Long.parseLong(storedLastModifiedTuple.getKey()) < currentTime){ + workingMap.put(ETAG + ":" + url, currentTime+":"+receivedEtag.getValue()); + changed = true; + } + } + + if(changed) { + replaceSucceeded = stateManager.replace(oldValue, workingMap, Scope.LOCAL); + } else { + break; + } + } + } catch (final IOException ioe) { + throw new ProcessException(ioe); + } + } + + protected static Tuple parseStateValue(String mapValue){ + int indexOfColon = mapValue.indexOf(":"); + + String timestamp = mapValue.substring(0,indexOfColon); + String value = mapValue.substring(indexOfColon+1); + return new Tuple<>(timestamp,value); + } } diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListFile.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListFile.java index c9f4482a99..e82be2c48d 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListFile.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListFile.java @@ -45,6 +45,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Pattern; import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.Stateful; import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; import org.apache.nifi.annotation.behavior.TriggerSerially; import org.apache.nifi.annotation.behavior.WritesAttribute; @@ -53,7 +54,9 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.state.Scope; import org.apache.nifi.flowfile.attributes.CoreAttributes; import org.apache.nifi.processor.DataUnit; import org.apache.nifi.processor.ProcessContext; @@ -92,7 +95,14 @@ import org.apache.nifi.processors.standard.util.FileInfo; "rw-rw-r--") }) @SeeAlso({GetFile.class, PutFile.class, FetchFile.class}) +@Stateful(scopes = {Scope.LOCAL, Scope.CLUSTER}, description = "After performing a listing of files, the timestamp of the newest file is stored, " + + "along with the filenames of all files that share that same timestamp. This allows the Processor to list only files that have been added or modified after " + + "this date the next time that the Processor is run. Whether the state is stored with a Local or Cluster scope depends on the value of the " + + " property.") public class ListFile extends AbstractListProcessor { + static final AllowableValue LOCATION_LOCAL = new AllowableValue("Local", "Local", "Input Directory is located on a local disk. State will be stored locally on each node in the cluster."); + static final AllowableValue LOCATION_REMOTE = new AllowableValue("Remote", "Remote", "Input Directory is located on a remote system. State will be stored across the cluster so that " + + "the listing can be performed on Primary Node Only and another node can pick up where the last node left off, if the Primary Node changes"); public static final PropertyDescriptor DIRECTORY = new PropertyDescriptor.Builder() .name("Input Directory") @@ -110,6 +120,14 @@ public class ListFile extends AbstractListProcessor { .defaultValue("true") .build(); + public static final PropertyDescriptor DIRECTORY_LOCATION = new PropertyDescriptor.Builder() + .name("Input Directory Location") + .description("Specifies where the Input Directory is located. This is used to determine whether state should be stored locally or across the cluster.") + .allowableValues(LOCATION_LOCAL, LOCATION_REMOTE) + .defaultValue(LOCATION_LOCAL.getValue()) + .required(true) + .build(); + public static final PropertyDescriptor FILE_FILTER = new PropertyDescriptor.Builder() .name("File Filter") .description("Only files whose names match the given regular expression will be picked up") @@ -182,6 +200,7 @@ public class ListFile extends AbstractListProcessor { final List properties = new ArrayList<>(); properties.add(DIRECTORY); properties.add(RECURSE); + properties.add(DIRECTORY_LOCATION); properties.add(FILE_FILTER); properties.add(PATH_FILTER); properties.add(MIN_AGE); @@ -274,6 +293,16 @@ public class ListFile extends AbstractListProcessor { return context.getProperty(DIRECTORY).evaluateAttributeExpressions().getValue(); } + @Override + protected Scope getStateScope(final ProcessContext context) { + final String location = context.getProperty(DIRECTORY_LOCATION).getValue(); + if (LOCATION_REMOTE.getValue().equalsIgnoreCase(location)) { + return Scope.CLUSTER; + } + + return Scope.LOCAL; + } + @Override protected List performListing(final ProcessContext context, final Long minTimestamp) throws IOException { final File path = new File(getPath(context)); diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListSFTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListSFTP.java index 609b69368d..d92f398111 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListSFTP.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListSFTP.java @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.List; import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.Stateful; import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; import org.apache.nifi.annotation.behavior.TriggerSerially; import org.apache.nifi.annotation.behavior.WritesAttribute; @@ -29,6 +30,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.state.Scope; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processors.standard.util.FileTransfer; import org.apache.nifi.processors.standard.util.SFTPTransfer; @@ -49,6 +51,10 @@ import org.apache.nifi.processors.standard.util.SFTPTransfer; @WritesAttribute(attribute = "filename", description = "The name of the file on the SFTP Server"), @WritesAttribute(attribute = "path", description = "The fully qualified name of the directory on the SFTP Server from which the file was pulled"), }) +@Stateful(scopes = {Scope.CLUSTER}, description = "After performing a listing of files, the timestamp of the newest file is stored, " + + "along with the filename all files that share that same timestamp. This allows the Processor to list only files that have been added or modified after " + + "this date the next time that the Processor is run. State is stored across the cluster so that this Processor can be run on Primary Node only and if " + + "a new Primary Node is selected, the new node will not duplicate the data that was listed by the previous Primary Node.") public class ListSFTP extends ListFileTransfer { @Override @@ -85,4 +91,11 @@ public class ListSFTP extends ListFileTransfer { protected String getProtocolName() { return "sftp"; } + + @Override + protected Scope getStateScope(final ProcessContext context) { + // Use cluster scope so that component can be run on Primary Node Only and can still + // pick up where it left off, even if the Primary Node changes. + return Scope.CLUSTER; + } } diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java index 626f7a44ae..b3ece4b908 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java @@ -19,11 +19,9 @@ package org.apache.nifi.processors.standard; import java.io.BufferedOutputStream; import java.io.DataInputStream; -import java.io.DataOutputStream; import java.io.File; import java.io.FileInputStream; import java.io.FileNotFoundException; -import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -48,6 +46,7 @@ import java.util.zip.Checksum; import org.apache.commons.lang3.StringUtils; import org.apache.nifi.annotation.behavior.InputRequirement; import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.behavior.Stateful; import org.apache.nifi.annotation.behavior.TriggerSerially; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; @@ -55,6 +54,8 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateMap; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; import org.apache.nifi.processor.AbstractProcessor; @@ -79,8 +80,14 @@ import org.apache.nifi.util.LongHolder; + "was not running (provided that the data still exists upon restart of NiFi). It is generally advisable to set the Run Schedule to a few seconds, rather than running " + "with the default value of 0 secs, as this Processor will consume a lot of resources if scheduled very aggressively. At this time, this Processor does not support " + "ingesting files that have been compressed when 'rolled over'.") +@Stateful(scopes = {Scope.LOCAL, Scope.CLUSTER}, description = "Stores state about where in the Tailed File it left off so that on restart it does not have to duplicate data. " + + "State is stored either local or clustered depend on the property.") public class TailFile extends AbstractProcessor { + static final AllowableValue LOCATION_LOCAL = new AllowableValue("Local", "Local", "File is located on a local disk drive. Each node in a cluster will tail a different file."); + static final AllowableValue LOCATION_REMOTE = new AllowableValue("Remote", "Remote", "File is located on a remote resource. This Processor will store state across the cluster so that " + + "it can be run on Primary Node Only and a new Primary Node can pick up where the last one left off."); + static final AllowableValue START_BEGINNING_OF_TIME = new AllowableValue("Beginning of Time", "Beginning of Time", "Start with the oldest data that matches the Rolling Filename Pattern and then begin reading from the File to Tail"); static final AllowableValue START_CURRENT_FILE = new AllowableValue("Beginning of File", "Beginning of File", @@ -104,12 +111,19 @@ public class TailFile extends AbstractProcessor { .expressionLanguageSupported(false) .required(false) .build(); + static final PropertyDescriptor FILE_LOCATION = new PropertyDescriptor.Builder() + .name("File Location") + .description("Specifies where the file is located, so that state can be stored appropriately in order to ensure that all data is consumed without duplicating data upon restart of NiFi") + .required(true) + .allowableValues(LOCATION_LOCAL, LOCATION_REMOTE) + .defaultValue(LOCATION_LOCAL.getValue()) + .build(); static final PropertyDescriptor STATE_FILE = new PropertyDescriptor.Builder() .name("State File") .description("Specifies the file that should be used for storing state about what data has been ingested so that upon restart NiFi can resume from where it left off") .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .expressionLanguageSupported(false) - .required(true) + .required(false) .build(); static final PropertyDescriptor START_POSITION = new PropertyDescriptor.Builder() .name("Initial Start Position") @@ -134,8 +148,9 @@ public class TailFile extends AbstractProcessor { final List properties = new ArrayList<>(); properties.add(FILENAME); properties.add(ROLLING_FILENAME_PATTERN); - properties.add(new PropertyDescriptor.Builder().fromPropertyDescriptor(STATE_FILE).defaultValue("./conf/state/" + getIdentifier()).build()); + properties.add(STATE_FILE); properties.add(START_POSITION); + properties.add(FILE_LOCATION); return properties; } @@ -146,18 +161,50 @@ public class TailFile extends AbstractProcessor { @Override public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) { - if (FILENAME.equals(descriptor)) { + if (isConfigurationRestored() && FILENAME.equals(descriptor)) { state = new TailFileState(newValue, null, null, 0L, 0L, null, ByteBuffer.allocate(65536)); tailFileChanged = true; } } + @OnScheduled public void recoverState(final ProcessContext context) throws IOException { - final String tailFilename = context.getProperty(FILENAME).getValue(); - final String stateFilename = context.getProperty(STATE_FILE).getValue(); + // Before the State Manager existed, we had to store state in a local file. Now, we want to use the State Manager + // instead. So we will need to recover the state that is stored in the file (if any), and then store that in our + // State Manager. But we do this only if nothing has ever been stored in the State Manager. + final Scope scope = getStateScope(context); + final StateMap stateMap = context.getStateManager().getState(scope); + if (stateMap.getVersion() == -1L) { + // State has never been stored in the State Manager. Try to recover state from a file, if one exists. + final Map stateFromFile = recoverStateValuesFromFile(context); + if (!stateFromFile.isEmpty()) { + persistState(stateFromFile, context); + recoverState(context, stateFromFile); + } + return; + } + + recoverState(context, stateMap.toMap()); + } + + /** + * Recovers values for the State that was stored in a local file. + * + * @param context the ProcessContext that indicates where the state is stored + * @return a Map that contains the keys defined in {@link TailFileState.StateKeys} + * @throws IOException if the state file exists but was unable to be read + */ + private Map recoverStateValuesFromFile(final ProcessContext context) throws IOException { + final String stateFilename = context.getProperty(STATE_FILE).getValue(); + if (stateFilename == null) { + return Collections.emptyMap(); + } + + final Map stateValues = new HashMap<>(4); final File stateFile = new File(stateFilename); + try (final FileInputStream fis = new FileInputStream(stateFile); final DataInputStream dis = new DataInputStream(fis)) { @@ -171,62 +218,109 @@ public class TailFile extends AbstractProcessor { long position = dis.readLong(); final long timestamp = dis.readLong(); final boolean checksumPresent = dis.readBoolean(); + final Long checksumValue; - FileChannel reader = null; - File tailFile = null; - - if (checksumPresent && tailFilename.equals(filename)) { - expectedRecoveryChecksum = dis.readLong(); - - // We have an expected checksum and the currently configured filename is the same as the state file. - // We need to check if the existing file is the same as the one referred to in the state file based on - // the checksum. - final Checksum checksum = new CRC32(); - final File existingTailFile = new File(filename); - if (existingTailFile.length() >= position) { - try (final InputStream tailFileIs = new FileInputStream(existingTailFile); - final CheckedInputStream in = new CheckedInputStream(tailFileIs, checksum)) { - StreamUtils.copy(in, new NullOutputStream(), state.getPosition()); - - final long checksumResult = in.getChecksum().getValue(); - if (checksumResult == expectedRecoveryChecksum) { - // Checksums match. This means that we want to resume reading from where we left off. - // So we will populate the reader object so that it will be used in onTrigger. If the - // checksums do not match, then we will leave the reader object null, so that the next - // call to onTrigger will result in a new Reader being created and starting at the - // beginning of the file. - getLogger().debug("When recovering state, checksum of tailed file matches the stored checksum. Will resume where left off."); - tailFile = existingTailFile; - reader = FileChannel.open(tailFile.toPath(), StandardOpenOption.READ); - getLogger().debug("Created FileChannel {} for {} in recoverState", new Object[] {reader, tailFile}); - - reader.position(position); - } else { - // we don't seek the reader to the position, so our reader will start at beginning of file. - getLogger().debug("When recovering state, checksum of tailed file does not match the stored checksum. Will begin tailing current file from beginning."); - } - } - } else { - // fewer bytes than our position, so we know we weren't already reading from this file. Keep reader at a position of 0. - getLogger().debug("When recovering state, existing file to tail is only {} bytes but position flag is {}; " - + "this indicates that the file has rotated. Will begin tailing current file from beginning.", new Object[] {existingTailFile.length(), position}); - } - - state = new TailFileState(tailFilename, tailFile, reader, position, timestamp, checksum, ByteBuffer.allocate(65536)); + if (checksumPresent) { + checksumValue = dis.readLong(); } else { - // If filename changed or there is no checksum present, then we have no expected checksum to use for recovery. - expectedRecoveryChecksum = null; - - // tailing a new file since the state file was written out. We will reset state. - state = new TailFileState(tailFilename, null, null, 0L, 0L, null, ByteBuffer.allocate(65536)); + checksumValue = null; } - getLogger().debug("Recovered state {}", new Object[] {state}); + stateValues.put(TailFileState.StateKeys.FILENAME, filename); + stateValues.put(TailFileState.StateKeys.POSITION, String.valueOf(position)); + stateValues.put(TailFileState.StateKeys.TIMESTAMP, String.valueOf(timestamp)); + stateValues.put(TailFileState.StateKeys.CHECKSUM, checksumValue == null ? null : String.valueOf(checksumValue)); } else { // encoding Version == -1... no data in file. Just move on. } } catch (final FileNotFoundException fnfe) { } + + return stateValues; + } + + + /** + * Updates member variables to reflect the "expected recovery checksum" and seek to the appropriate location in the + * tailed file, updating our checksum, so that we are ready to proceed with the {@link #onTrigger(ProcessContext, ProcessSession)} call. + * + * @param context the ProcessContext + * @param stateValues the values that were recovered from state that was previously stored. This Map should be populated with the keys defined + * in {@link TailFileState.StateKeys}. + * @throws IOException if unable to seek to the appropriate location in the tailed file. + */ + private void recoverState(final ProcessContext context, final Map stateValues) throws IOException { + if (stateValues == null) { + return; + } + + if (!stateValues.containsKey(TailFileState.StateKeys.FILENAME)) { + return; + } + if (!stateValues.containsKey(TailFileState.StateKeys.POSITION)) { + return; + } + if (!stateValues.containsKey(TailFileState.StateKeys.TIMESTAMP)) { + return; + } + + final String currentFilename = context.getProperty(FILENAME).getValue(); + final String checksumValue = stateValues.get(TailFileState.StateKeys.CHECKSUM); + final boolean checksumPresent = (checksumValue != null); + final String storedStateFilename = stateValues.get(TailFileState.StateKeys.FILENAME); + final long position = Long.parseLong(stateValues.get(TailFileState.StateKeys.POSITION)); + final long timestamp = Long.parseLong(stateValues.get(TailFileState.StateKeys.TIMESTAMP)); + + FileChannel reader = null; + File tailFile = null; + + if (checksumPresent && currentFilename.equals(storedStateFilename)) { + expectedRecoveryChecksum = Long.parseLong(checksumValue); + + // We have an expected checksum and the currently configured filename is the same as the state file. + // We need to check if the existing file is the same as the one referred to in the state file based on + // the checksum. + final Checksum checksum = new CRC32(); + final File existingTailFile = new File(storedStateFilename); + if (existingTailFile.length() >= position) { + try (final InputStream tailFileIs = new FileInputStream(existingTailFile); + final CheckedInputStream in = new CheckedInputStream(tailFileIs, checksum)) { + StreamUtils.copy(in, new NullOutputStream(), state.getPosition()); + + final long checksumResult = in.getChecksum().getValue(); + if (checksumResult == expectedRecoveryChecksum) { + // Checksums match. This means that we want to resume reading from where we left off. + // So we will populate the reader object so that it will be used in onTrigger. If the + // checksums do not match, then we will leave the reader object null, so that the next + // call to onTrigger will result in a new Reader being created and starting at the + // beginning of the file. + getLogger().debug("When recovering state, checksum of tailed file matches the stored checksum. Will resume where left off."); + tailFile = existingTailFile; + reader = FileChannel.open(tailFile.toPath(), StandardOpenOption.READ); + getLogger().debug("Created FileChannel {} for {} in recoverState", new Object[] {reader, tailFile}); + + reader.position(position); + } else { + // we don't seek the reader to the position, so our reader will start at beginning of file. + getLogger().debug("When recovering state, checksum of tailed file does not match the stored checksum. Will begin tailing current file from beginning."); + } + } + } else { + // fewer bytes than our position, so we know we weren't already reading from this file. Keep reader at a position of 0. + getLogger().debug("When recovering state, existing file to tail is only {} bytes but position flag is {}; " + + "this indicates that the file has rotated. Will begin tailing current file from beginning.", new Object[] {existingTailFile.length(), position}); + } + + state = new TailFileState(currentFilename, tailFile, reader, position, timestamp, checksum, ByteBuffer.allocate(65536)); + } else { + // If filename changed or there is no checksum present, then we have no expected checksum to use for recovery. + expectedRecoveryChecksum = null; + + // tailing a new file since the state file was written out. We will reset state. + state = new TailFileState(currentFilename, null, null, 0L, 0L, null, ByteBuffer.allocate(65536)); + } + + getLogger().debug("Recovered state {}", new Object[] {state}); } @@ -572,40 +666,27 @@ public class TailFile extends AbstractProcessor { } + private Scope getStateScope(final ProcessContext context) { + final String location = context.getProperty(FILE_LOCATION).getValue(); + if (LOCATION_REMOTE.getValue().equalsIgnoreCase(location)) { + return Scope.CLUSTER; + } + + return Scope.LOCAL; + } private void persistState(final TailFileState state, final ProcessContext context) { - final String stateFilename = context.getProperty(STATE_FILE).getValue(); + persistState(state.toStateMap(), context); + } + + private void persistState(final Map state, final ProcessContext context) { try { - persistState(state, stateFilename); + context.getStateManager().setState(state, getStateScope(context)); } catch (final IOException e) { - getLogger().warn("Failed to update state file {} due to {}; some data may be duplicated on restart of NiFi", new Object[] {stateFilename, e}); + getLogger().warn("Failed to store state due to {}; some data may be duplicated on restart of NiFi", new Object[] {e}); } } - private void persistState(final TailFileState state, final String stateFilename) throws IOException { - getLogger().debug("Persisting state {} to {}", new Object[] {state, stateFilename}); - - final File stateFile = new File(stateFilename); - File directory = stateFile.getParentFile(); - if (directory != null && !directory.exists() && !directory.mkdirs()) { - getLogger().warn("Failed to persist state to {} because the parent directory does not exist and could not be created. This may result in data being duplicated upon restart of NiFi"); - return; - } - try (final FileOutputStream fos = new FileOutputStream(stateFile); - final DataOutputStream dos = new DataOutputStream(fos)) { - - dos.writeInt(0); // version - dos.writeUTF(state.getFilename()); - dos.writeLong(state.getPosition()); - dos.writeLong(state.getTimestamp()); - if (state.getChecksum() == null) { - dos.writeBoolean(false); - } else { - dos.writeBoolean(true); - dos.writeLong(state.getChecksum().getValue()); - } - } - } private FileChannel createReader(final File file, final long position) { final FileChannel reader; @@ -729,7 +810,7 @@ public class TailFile extends AbstractProcessor { // must ensure that we do session.commit() before persisting state in order to avoid data loss. session.commit(); - persistState(state, context.getProperty(STATE_FILE).getValue()); + persistState(state, context); } } else { getLogger().debug("Checksum for {} did not match expected checksum. Checksum for file was {} but expected {}. Will consume entire file", @@ -801,6 +882,13 @@ public class TailFile extends AbstractProcessor { private final Checksum checksum; private final ByteBuffer buffer; + private static class StateKeys { + public static final String FILENAME = "filename"; + public static final String POSITION = "position"; + public static final String TIMESTAMP = "timestamp"; + public static final String CHECKSUM = "checksum"; + } + public TailFileState(final String filename, final File file, final FileChannel reader, final long position, final long timestamp, final Checksum checksum, final ByteBuffer buffer) { this.filename = filename; this.file = file; @@ -843,5 +931,14 @@ public class TailFile extends AbstractProcessor { public String toString() { return "TailFileState[filename=" + filename + ", position=" + position + ", timestamp=" + timestamp + ", checksum=" + (checksum == null ? "null" : checksum.getValue()) + "]"; } + + public Map toStateMap() { + final Map map = new HashMap<>(4); + map.put(StateKeys.FILENAME, filename); + map.put(StateKeys.POSITION, String.valueOf(position)); + map.put(StateKeys.TIMESTAMP, String.valueOf(timestamp)); + map.put(StateKeys.CHECKSUM, checksum == null ? null : String.valueOf(checksum.getValue())); + return map; + } } } 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..3a432e7bf4 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 @@ -18,6 +18,7 @@ package org.apache.nifi.processors.standard; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; import java.io.File; import java.io.IOException; @@ -26,8 +27,11 @@ 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.components.state.StateMap; import org.apache.nifi.controller.AbstractControllerService; import org.apache.nifi.distributed.cache.client.Deserializer; import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient; @@ -35,6 +39,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 +101,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 +114,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 @@ -126,6 +156,43 @@ public class TestAbstractListProcessor { assertEquals(1, cache.fetchCount); } + @Test + public void testOnlyNewStateStored() throws IOException { + final ConcreteListProcessor proc = new ConcreteListProcessor(); + final TestRunner runner = TestRunners.newTestRunner(proc); + runner.run(); + + runner.assertAllFlowFilesTransferred(ConcreteListProcessor.REL_SUCCESS, 0); + proc.addEntity("name", "id", 1492L); + proc.addEntity("name", "id2", 1492L); + + runner.run(); + runner.assertAllFlowFilesTransferred(ConcreteListProcessor.REL_SUCCESS, 2); + runner.clearTransferState(); + + final StateMap stateMap = runner.getStateManager().getState(Scope.CLUSTER); + assertEquals(1, stateMap.getVersion()); + + final Map map = stateMap.toMap(); + assertEquals(3, map.size()); + assertEquals("1492", map.get("timestamp")); + assertTrue(map.containsKey("id.1")); + assertTrue(map.containsKey("id.2")); + + proc.addEntity("new name", "new id", 1493L); + runner.run(); + + runner.assertAllFlowFilesTransferred(ConcreteListProcessor.REL_SUCCESS, 1); + final StateMap updatedStateMap = runner.getStateManager().getState(Scope.CLUSTER); + assertEquals(2, updatedStateMap.getVersion()); + + final Map updatedValues = updatedStateMap.toMap(); + assertEquals(2, updatedValues.size()); + assertEquals("1493", updatedValues.get("timestamp")); + assertEquals("new id", updatedValues.get("id.1")); + } + + private static class DistributedCache extends AbstractControllerService implements DistributedMapCacheClient { private final Map stored = new HashMap<>(); private int fetchCount = 0; @@ -174,7 +241,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) { @@ -217,5 +284,10 @@ public class TestAbstractListProcessor { protected boolean isListingResetNecessary(PropertyDescriptor property) { return false; } + + @Override + protected Scope getStateScope(final ProcessContext context) { + return Scope.CLUSTER; + } } } 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 54e6a29594..0d7394ec07 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 @@ -112,7 +104,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)); client.initialize(clientInitContext); return client; @@ -219,6 +211,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) { @@ -255,33 +248,4 @@ public class TestDetectDuplicate { exists = true; } } - - 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..f8e4122d37 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,32 +16,25 @@ */ 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; -import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + /** * */ @@ -55,23 +48,6 @@ public class TestGetHTTP { System.setProperty("org.slf4j.simpleLogger.showDateTime", "true"); System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.GetHTTP", "debug"); System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.TestGetHTTP", "debug"); - File confDir = new File("conf"); - if (!confDir.exists()) { - confDir.mkdir(); - } - } - - @AfterClass - public static void after() { - File confDir = new File("conf"); - assertTrue(confDir.exists()); - File[] files = confDir.listFiles(); - if (files.length > 0) { - for (File file : files) { - assertTrue("Failed to delete " + file.getName(), file.delete()); - } - } - assertTrue(confDir.delete()); } @Test @@ -97,26 +73,26 @@ 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+":"+destination, "", Scope.LOCAL); + controller.getStateManager().assertStateNotEquals(GetHTTP.LAST_MODIFIED+":"+destination, "Thu, 01 Jan 1970 00:00:00 GMT", Scope.LOCAL); + // ran twice, but got one...which is good controller.assertTransferCount(GetHTTP.REL_SUCCESS, 1); // verify remote.source flowfile attribute controller.getFlowFilesForRelationship(GetHTTP.REL_SUCCESS).get(0).assertAttributeEquals("gethttp.remote.source", "localhost"); - controller.clearTransferState(); // turn off checking for etag and lastModified RESTServiceContentModified.IGNORE_ETAG = true; RESTServiceContentModified.IGNORE_LAST_MODIFIED = true; controller.run(2); + // ran twice, got two...which is good controller.assertTransferCount(GetHTTP.REL_SUCCESS, 2); controller.clearTransferState(); @@ -139,110 +115,95 @@ public class TestGetHTTP { RESTServiceContentModified.IGNORE_ETAG = false; RESTServiceContentModified.ETAG = 1; controller.run(2); + // ran twice, got 1...but should have new cached etag controller.assertTransferCount(GetHTTP.REL_SUCCESS, 1); - assertEquals("1", getHTTPProcessor.entityTagRef.get()); + String eTagStateValue = controller.getStateManager().getState(Scope.LOCAL).get(GetHTTP.ETAG+":"+destination); + assertEquals("1",GetHTTP.parseStateValue(eTagStateValue).getValue()); 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+":"+destination); 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+":"+destination, lastMod, Scope.LOCAL); controller.clearTransferState(); - // shutdown web service } finally { + // shutdown web service server.shutdownServer(); } } + @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()); - } + public final void testContentModifiedTwoServers() throws Exception { + // set up web services + ServletHandler handler1 = new ServletHandler(); + handler1.addServletWithMapping(RESTServiceContentModified.class, "/*"); - // set up web service - ServletHandler handler = new ServletHandler(); - handler.addServletWithMapping(RESTServiceContentModified.class, "/*"); + ServletHandler handler2 = new ServletHandler(); + handler2.addServletWithMapping(RESTServiceContentModified.class, "/*"); - // create the service - TestServer server = new TestServer(); - server.addHandler(handler); + // create the services + TestServer server1 = new TestServer(); + server1.addHandler(handler1); + + TestServer server2 = new TestServer(); + server2.addHandler(handler2); try { - server.startServer(); + server1.startServer(); + server2.startServer(); - // get the server url - String destination = server.getUrl(); + // this is the base urls with the random ports + String destination1 = server1.getUrl(); + String destination2 = server2.getUrl(); // set up NiFi mock controller controller = TestRunners.newTestRunner(GetHTTP.class); controller.setProperty(GetHTTP.CONNECTION_TIMEOUT, "5 secs"); + controller.setProperty(GetHTTP.URL, destination1); 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.getStateManager().assertStateNotSet(GetHTTP.ETAG+":"+destination1, Scope.LOCAL); + controller.getStateManager().assertStateNotSet(GetHTTP.LAST_MODIFIED+":"+destination1, Scope.LOCAL); 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)); + controller.getStateManager().assertStateNotEquals(GetHTTP.ETAG+":"+destination1, "", Scope.LOCAL); + controller.getStateManager().assertStateNotEquals(GetHTTP.LAST_MODIFIED+":"+destination1, "Thu, 01 Jan 1970 00:00:00 GMT", Scope.LOCAL); + // 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)); + controller.setProperty(GetHTTP.URL, destination2); + controller.getStateManager().assertStateNotSet(GetHTTP.ETAG+":"+destination2, Scope.LOCAL); + controller.getStateManager().assertStateNotSet(GetHTTP.LAST_MODIFIED+":"+destination2, Scope.LOCAL); - 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()); + // ran twice, but got one...which is good + controller.assertTransferCount(GetHTTP.REL_SUCCESS, 1); + + // verify the lastModified's and entityTags are updated + controller.getStateManager().assertStateNotEquals(GetHTTP.ETAG+":"+destination1, "", Scope.LOCAL); + controller.getStateManager().assertStateNotEquals(GetHTTP.LAST_MODIFIED+":"+destination1, "Thu, 01 Jan 1970 00:00:00 GMT", Scope.LOCAL); + controller.getStateManager().assertStateNotEquals(GetHTTP.ETAG+":"+destination2, "", Scope.LOCAL); + controller.getStateManager().assertStateNotEquals(GetHTTP.LAST_MODIFIED+":"+destination2, "Thu, 01 Jan 1970 00:00:00 GMT", Scope.LOCAL); - // shutdown web service } finally { - server.shutdownServer(); + // shutdown web services + server1.shutdownServer(); + server2.shutdownServer(); } } diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestListFile.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestListFile.java index 7111686caa..93319d097c 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestListFile.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestListFile.java @@ -86,20 +86,6 @@ public class TestListFile { } } - @Test - public void testGetSupportedPropertyDescriptors() throws Exception { - List properties = processor.getSupportedPropertyDescriptors(); - assertEquals(9, properties.size()); - assertEquals(ListFile.DIRECTORY, properties.get(0)); - assertEquals(ListFile.RECURSE, properties.get(1)); - assertEquals(ListFile.FILE_FILTER, properties.get(2)); - assertEquals(ListFile.PATH_FILTER, properties.get(3)); - assertEquals(ListFile.MIN_AGE, properties.get(4)); - assertEquals(ListFile.MAX_AGE, properties.get(5)); - assertEquals(ListFile.MIN_SIZE, properties.get(6)); - assertEquals(ListFile.MAX_SIZE, properties.get(7)); - assertEquals(ListFile.IGNORE_HIDDEN_FILES, properties.get(8)); - } @Test public void testGetRelationships() 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..66dc854b9a 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(proc)); 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(proc)); final ValidationResult validationResult = ctx.setProperty("RouteA", "${a:length()"); // Should be boolean assertFalse(validationResult.isValid()); } diff --git a/pom.xml b/pom.xml index d535ae26f7..88558ca280 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 @@ -754,10 +755,30 @@ language governing permissions and limitations under the License. --> 1.3.1
+ org.apache.zookeeper + zookeeper + 3.4.6 + + + + + org.apache.curator + curator-test + 2.9.1 + test + + + org.testng + testng + 6.8.8 + test + + org.jsoup jsoup 1.8.3 + org.apache.nifi nifi-api @@ -1322,71 +1343,100 @@ language governing permissions and limitations under the License. --> - + - - + + - - + + - + - + - + - - + + - - + + - - + + - + - + - - + + - + - - + + - - - - - - + + + + + + @@ -1485,12 +1535,12 @@ language governing permissions and limitations under the License. --> - + disable-doclint 1.8