From b86f144eed8baa70eb65cc0b4ade3f4fb0e215aa Mon Sep 17 00:00:00 2001 From: Koji Kawamura Date: Sat, 30 Jul 2016 11:18:37 +0900 Subject: [PATCH 1/3] NIFI-2078, 2363, 2364: State management - Addressed review feedback - Updated java doc, and dev-guide - Added assertion code to StateManager implementations so that it throws IllegalArgumentException if Scope.EXTERNAL is passed NIFI-2078: State managed externally - Added EXTERNAL Scope and corresponding ExternalStateManager interface to handle components' state managed. - Added UI codes to display external state - Added view/clear functionality to ConsumeKafka - Added view/clear functionality to GetKafka - Capture property value change, so that external state can be accessed before onTrigger is called - Tested with Kerberized Zk and Kafka. NIFI-2363: CLUSTER scope should be managed on primary node - Added endpoint merge logic for getting CLUSTER and EXTERNAL state - Added endpoint merger for clear state - Refactered NiFiServiceFacade -> controllerService, processor, reportingTask DAO -> componentStateDAO -> StateManager chain. To make it more simple and cluster aware. NIFI-2364: Avoid being disconnected by error related to external system - Added ClearComponentStateResultEntity to represent a result of clear state operation without throwing exceptions. - Added return for clearXXXState at NiFiServiceFacade. - Changed response type to ClearComponentStateResultEntity for controllerService, processor, reportingTask resources. The response was not used at js before. Misc: - Fixed typo in ContentViewerController --- .../nifi/annotation/behavior/Stateful.java | 9 +- .../state/ExternalStateManager.java | 90 ++++ .../apache/nifi/components/state/Scope.java | 11 +- .../components}/state/StandardStateMap.java | 3 +- .../nifi/components/state/StateManager.java | 8 + .../src/main/asciidoc/developer-guide.adoc | 40 +- .../apache/nifi/state/MockStateManager.java | 4 + .../nifi/web/api/dto/ComponentStateDTO.java | 15 + .../ClearComponentStateResultEntity.java | 46 ++ .../http/StandardHttpResponseMerger.java | 2 + .../ClearComponentStateEndpointMerger.java | 71 +++ .../ComponentStateEndpointMerger.java | 94 +++- .../TestComponentStateEndpointMerger.java | 437 ++++++++++++++++++ .../state/StandardStateManager.java | 4 + .../nifi/controller/state/StateMapSerDe.java | 1 + .../local/WriteAheadLocalStateProvider.java | 2 +- .../zookeeper/ZooKeeperStateProvider.java | 2 +- .../controller/state/TestStateMapSerDe.java | 1 + .../apache/nifi/web/NiFiServiceFacade.java | 10 +- .../nifi/web/StandardNiFiServiceFacade.java | 103 ++++- .../web/api/ControllerServiceResource.java | 11 +- .../nifi/web/api/ProcessorResource.java | 13 +- .../nifi/web/api/ReportingTaskResource.java | 11 +- .../apache/nifi/web/api/dto/DtoFactory.java | 13 +- .../nifi/web/controller/ControllerFacade.java | 12 +- .../nifi/web/dao/ComponentStateDAO.java | 46 +- .../nifi/web/dao/ControllerServiceDAO.java | 17 - .../org/apache/nifi/web/dao/ProcessorDAO.java | 17 - .../apache/nifi/web/dao/ReportingTaskDAO.java | 17 - .../dao/impl/StandardComponentStateDAO.java | 82 ++-- .../impl/StandardControllerServiceDAO.java | 20 - .../web/dao/impl/StandardProcessorDAO.java | 21 +- .../dao/impl/StandardReportingTaskDAO.java | 19 - .../main/resources/nifi-web-api-context.xml | 4 +- .../nifi/web/ContentViewerController.java | 2 +- .../webapp/js/nf/canvas/nf-component-state.js | 95 ++-- .../nifi/processors/kafka/GetKafka.java | 83 +++- .../nifi/processors/kafka/KafkaUtils.java | 123 ++++- .../nifi/processors/kafka/TestGetKafka.java | 44 ++ .../nifi-kafka-pubsub-processors/pom.xml | 4 + .../kafka/pubsub/AbstractKafkaProcessor.java | 37 +- .../processors/kafka/pubsub/ConsumeKafka.java | 263 ++++++++++- .../kafka/pubsub/ConsumeKafkaTest.java | 86 +++- 43 files changed, 1634 insertions(+), 359 deletions(-) create mode 100644 nifi-api/src/main/java/org/apache/nifi/components/state/ExternalStateManager.java rename {nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller => nifi-api/src/main/java/org/apache/nifi/components}/state/StandardStateMap.java (94%) create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ClearComponentStateResultEntity.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ClearComponentStateEndpointMerger.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/endpoints/TestComponentStateEndpointMerger.java 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 index de32bd7d74c6..324d18ac6d9c 100644 --- 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 @@ -26,11 +26,16 @@ import org.apache.nifi.components.state.Scope; import org.apache.nifi.components.state.StateManager; +import org.apache.nifi.components.state.ExternalStateManager; /** *

* 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 + * that the component makes use of the {@link StateManager} or implements + * {@link ExternalStateManager}. + *

+ *

+ * 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 @@ -43,7 +48,7 @@ @Inherited public @interface Stateful { /** - * Provides a description of what information is being stored in the {@link StateManager} + * Provides a description of what information is being stored in the {@link StateManager} or {@link ExternalStateManager} */ String description(); diff --git a/nifi-api/src/main/java/org/apache/nifi/components/state/ExternalStateManager.java b/nifi-api/src/main/java/org/apache/nifi/components/state/ExternalStateManager.java new file mode 100644 index 000000000000..896c3e349010 --- /dev/null +++ b/nifi-api/src/main/java/org/apache/nifi/components/state/ExternalStateManager.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.components.state; + +import org.apache.nifi.annotation.behavior.Stateful; + +import java.io.IOException; + +/** + *

+ * The ExternalStateManager is responsible for providing NiFi a mechanism for retrieving + * and clearing state stored in an external system a NiFi component interact with. + *

+ * + *

+ * This mechanism is designed to allow Data Flow Managers to easily view and clear + * component state stored externally. + * It is advisable to implement this interface for components those have such state. + *

+ * + *

+ * When calling methods in this class, the state is always retrieved/cleared from external system + * regardless NiFi instance is a part of a cluster or standalone. + *

+ * + *

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

+ * + */ +public interface ExternalStateManager { + + /** + * Returns the current state for the component. This return value may be null. + * + * @return the current state for the component or null if there is no state is retrieved + * @throws IOException if unable to communicate with the underlying storage mechanism + */ + StateMap getExternalState() throws IOException; + + /** + * Clears all keys and values from the component's state + * + * @throws IOException if unable to communicate with the underlying storage mechanism + */ + void clearExternalState() throws IOException; + + /** + *

+ * In a clustered environment, implementations of {@link ExternalStateManager} interface is called based on the + * {@link ExternalStateScope} value returned by its {@link #getExternalStateScope()} method. + *

+ */ + enum ExternalStateScope { + /** + * 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 per node. I.e., the same component will have different state in the external system for + * each node in the cluster. + */ + NODE + } + + /** + * @return An external state scope defining how this state manager should behave. + */ + ExternalStateScope getExternalStateScope(); + +} 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 index dd0d0aa40f95..53eb2e2d73ed 100644 --- 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 @@ -32,7 +32,16 @@ public enum Scope { * 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; + LOCAL, + + /** + *

+ * State is to be treated local to the node but managed by external system. + * It can also be treated as "global" across the cluster. + * See {@link ExternalStateManager#getExternalStateScope()} for detail. + *

+ */ + EXTERNAL; @Override public String toString() { 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-api/src/main/java/org/apache/nifi/components/state/StandardStateMap.java similarity index 94% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/StandardStateMap.java rename to nifi-api/src/main/java/org/apache/nifi/components/state/StandardStateMap.java index 672fd6f5efad..23962d4e4c3f 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/StandardStateMap.java +++ b/nifi-api/src/main/java/org/apache/nifi/components/state/StandardStateMap.java @@ -15,12 +15,11 @@ * limitations under the License. */ -package org.apache.nifi.controller.state; +package org.apache.nifi.components.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; 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 index 768f77317bcf..184dcb1ba62d 100644 --- 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 @@ -52,6 +52,14 @@ * 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. *

+ * + *

+ * Some external systems may store NiFi component state when NiFi components interact with those, + * and those state are scoped as {@link Scope#EXTERNAL}. + * A component that wishes to provide Data Flow Managers access to external state, should implement {@link ExternalStateManager}. + * Since StateManager doesn't manage external states, implementation of StateManager class + * should throw {@link IllegalArgumentException} when {@link Scope#EXTERNAL} is passed as the scope argument of each method. + *

*/ public interface StateManager { diff --git a/nifi-docs/src/main/asciidoc/developer-guide.adoc b/nifi-docs/src/main/asciidoc/developer-guide.adoc index d7efde49c2e5..84605fdfb54b 100644 --- a/nifi-docs/src/main/asciidoc/developer-guide.adoc +++ b/nifi-docs/src/main/asciidoc/developer-guide.adoc @@ -579,32 +579,35 @@ replaced. Moreover, the only implementation that is currently supported for stor backed by ZooKeeper. As such, the entire State Map must be less than 1 MB in size, after being serialized. Attempting to store more than this will result in an Exception being thrown. If the interactions required by the Processor for managing state are more complex than this (e.g., large amounts of data must be stored -and retrieved, or individual keys must be stored and fetched individually) than a different mechanism should +and retrieved, or individual keys must be stored and fetched individually) then a different mechanism should be used (e.g., communicating with an external database). [[state_scope]] ==== Scope When communicating with the State Manager, all method calls require that a Scope be provided. This Scope will -either be `Scope.NODE` or `Scope.CLUSTER`. If NiFi is run in a cluster, this Scope provides important information +either be `Scope.LOCAL` 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. If state is stored and retrieved using `Scope.NODE`, then each node will see a different +state storage mechanism. If state is stored and retrieved using `Scope.LOCAL`, 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 +a scope of `Scope.LOCAL` 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. +There is also `Scope.EXTERNAL`, that is conceptually different than `Scope.LOCAL` or `Scope.CLUSTER`. +`Scope.EXTERNAL` represents that the state is managed by external system, not by NiFi State Manager. +See the <> for detail. ==== Storing and Retrieving State State is stored using the StateManager's `getState`, `setState`, `replace`, and `clear` methods. All of these methods require that a Scope be provided. It should be noted that the state that is stored with the Local scope is entirely different than state stored with a Cluster scope. 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` (unless a value was +and then attempts to retrieve the value using the `Scope.LOCAL` scope, the value retrieved will be `null` (unless a value was also stored with the same key using the `Scope.CLUSTER` scope). Each Processor's state, is stored in isolation from other Processors' state. @@ -613,6 +616,11 @@ necessary to share state between two Processors of different types, or two Proce 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. +==== Accessing state from NiFi UI + +When a DFM view or clear state from NiFi UI, all of the `Scope.LOCAL`, `Scope.CLUSTER`, and `Scope.EXTERNAL` state are retrieved or cleared at the same time. +To avoid accessing underlying data storage unnecessarily frequently, those operations for `Scope.CLUSTER` and `Scope.EXTERNAL` with `ExternalStateScope.CLUSTER` are executed only from a primary node in a NiFi cluster. + ==== Unit Tests NiFi's Mock Framework provides an extensive collection of tools to perform unit testing of Processors. Processor unit tests typically @@ -625,8 +633,30 @@ Additionally, the `MockStateManager` exposes a handful of `assert*` methods to p The `MockStateManager` also provides the ability to indicate that the unit test should immediately fail if state is updated for a particular `Scope`. +[[external_state_manager]] +==== External State Manager + +Some external system stores client state in it, so that multiple clients can share workloads or for any other reasons. +For example, Apache Kafka stores consumer group's offset per partition, to provide a mechanism for consumers to resume where they left off. +If a DFM wants NiFi ConsumeKafka processor to replay messages from beginning of a topic, the offset information stored at Kafka server needs to be cleared. +Also if a DFM can view the offset information from NiFi UI, it's useful to understand how far a consumer has processed. + +External State Manager is a mechanism to provide DFMs such interactions with external systems. +If a Processor, Controller Service or Reporting Task interact with an external system that manages NiFi component state, +it's advisable for those components to implement `ExternalStateManager` interface, and annotate the class with `@Stateful` using `Scope.EXTERNAL`. + +By doing so, NiFi UI lets a DFM to access component states by calling corresponding `getExternalState` or `clearExternalState` method. +To access an external system from those method implementations, you will need configured property values of the component, +such as Database connection URL or Kafka broker address. +Since `ExternalStateManager` methods can be called before component is fully configured or scheduled, +it's important to use `onPropertyModified` to capture those required configuration values for an external system connection. +`ExternalStateManager` also has the `getExternalStateScope` method which return either `ExternalStateScope.NODE` or `ExternalStateScope.CLUSTER` +to indicate how a NiFi cluster should access the target external system. +With `ExternalStateScope.NODE`, each node in a NiFi cluster attempts to retrieve or clear state from an external system +then individual responses will be merged into the final single response, +while those operations are executed only on a Primary Node with `ExternalStateScope.CLUSTER`. === Reporting Processor Activity diff --git a/nifi-mock/src/main/java/org/apache/nifi/state/MockStateManager.java b/nifi-mock/src/main/java/org/apache/nifi/state/MockStateManager.java index 81ad988ca904..c1b0828233ef 100644 --- a/nifi-mock/src/main/java/org/apache/nifi/state/MockStateManager.java +++ b/nifi-mock/src/main/java/org/apache/nifi/state/MockStateManager.java @@ -138,6 +138,10 @@ private void verifyCanGet(final Scope scope) throws IOException { } private void verifyAnnotation(final Scope scope) { + if (scope == Scope.EXTERNAL) { + Assert.fail("EXTERNAL scope is not managed by StateManager." + + " A component should implement ExternalStateManager by itself to support EXTERNAL scope state."); + } // 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 " 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 index 9036d5377616..2912810a4abd 100644 --- 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 @@ -30,6 +30,7 @@ public class ComponentStateDTO { private String stateDescription; private StateMapDTO clusterState; private StateMapDTO localState; + private StateMapDTO externalState; /** * @return The component identifier @@ -86,4 +87,18 @@ public StateMapDTO getLocalState() { public void setLocalState(StateMapDTO localState) { this.localState = localState; } + + /** + * @return The external state for this component + */ + @ApiModelProperty( + value = "The external state for this component." + ) + public StateMapDTO getExternalState() { + return externalState; + } + + public void setExternalState(StateMapDTO externalState) { + this.externalState = externalState; + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ClearComponentStateResultEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ClearComponentStateResultEntity.java new file mode 100644 index 000000000000..4cd2f4bbcf26 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ClearComponentStateResultEntity.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.web.api.entity; + +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 result of clear state operation. + */ +@XmlRootElement(name = "result") +public class ClearComponentStateResultEntity extends Entity { + + private boolean cleared = true; + private String message = null; + + public boolean isCleared() { + return cleared; + } + + public void setCleared(boolean cleared) { + this.cleared = cleared; + } + + public String getMessage() { + return message; + } + + public void setMessage(String message) { + this.message = message; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMerger.java index f5e7d98a6ecf..f28c591c3af2 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMerger.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMerger.java @@ -18,6 +18,7 @@ package org.apache.nifi.cluster.coordination.http; import org.apache.nifi.cluster.coordination.http.endpoints.BulletinBoardEndpointMerger; +import org.apache.nifi.cluster.coordination.http.endpoints.ClearComponentStateEndpointMerger; import org.apache.nifi.cluster.coordination.http.endpoints.ComponentStateEndpointMerger; import org.apache.nifi.cluster.coordination.http.endpoints.ConnectionEndpointMerger; import org.apache.nifi.cluster.coordination.http.endpoints.ConnectionStatusEndpiontMerger; @@ -102,6 +103,7 @@ public class StandardHttpResponseMerger implements HttpResponseMerger { endpointMergers.add(new DropRequestEndpiontMerger()); endpointMergers.add(new ListFlowFilesEndpointMerger()); endpointMergers.add(new ComponentStateEndpointMerger()); + endpointMergers.add(new ClearComponentStateEndpointMerger()); endpointMergers.add(new BulletinBoardEndpointMerger()); endpointMergers.add(new StatusHistoryEndpointMerger()); endpointMergers.add(new SystemDiagnosticsEndpointMerger()); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ClearComponentStateEndpointMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ClearComponentStateEndpointMerger.java new file mode 100644 index 000000000000..cdd9840824dc --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ClearComponentStateEndpointMerger.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.cluster.coordination.http.endpoints; + +import org.apache.nifi.cluster.manager.NodeResponse; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.web.api.entity.ClearComponentStateResultEntity; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.URI; +import java.util.Map; +import java.util.Set; +import java.util.regex.Pattern; + +public class ClearComponentStateEndpointMerger extends AbstractSingleEntityEndpoint { + public static final Pattern PROCESSOR_STATE_URI_PATTERN = Pattern.compile("/nifi-api/processors/[a-f0-9\\-]{36}/state/clear-requests"); + public static final Pattern CONTROLLER_SERVICE_STATE_URI_PATTERN = Pattern.compile("/nifi-api/controller-services/[a-f0-9\\-]{36}/state/clear-requests"); + public static final Pattern REPORTING_TASK_STATE_URI_PATTERN = Pattern.compile("/nifi-api/reporting-tasks/[a-f0-9\\-]{36}/state/clear-requests"); + + private static final Logger logger = LoggerFactory.getLogger(ClearComponentStateEndpointMerger.class); + + @Override + public boolean canHandle(URI uri, String method) { + if (!"POST".equalsIgnoreCase(method)) { + return false; + } + + return PROCESSOR_STATE_URI_PATTERN.matcher(uri.getPath()).matches() + || CONTROLLER_SERVICE_STATE_URI_PATTERN.matcher(uri.getPath()).matches() + || REPORTING_TASK_STATE_URI_PATTERN.matcher(uri.getPath()).matches(); + } + + @Override + protected Class getEntityClass() { + return ClearComponentStateResultEntity.class; + } + + @Override + protected void mergeResponses(ClearComponentStateResultEntity clientEntity, Map entityMap, Set successfulResponses, Set problematicResponses) { + + if (logger.isDebugEnabled()) { + logger.debug("entityMap={}, successfulResponse={}, problematicResponse={}", entityMap, successfulResponses, problematicResponses); + entityMap.forEach((id, res) -> logger.debug("nodeId={}, res.isCleared={}", id, res.isCleared())); + } + + // If there's a uncleared response, use that. + final ClearComponentStateResultEntity unclearedRes = entityMap.values().stream().filter(res -> !res.isCleared()).findFirst().orElse(null); + if (unclearedRes != null) { + clientEntity.setCleared(unclearedRes.isCleared()); + clientEntity.setMessage(unclearedRes.getMessage()); + } + + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ComponentStateEndpointMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ComponentStateEndpointMerger.java index 3b03ed0b1e31..84e6d706bf39 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ComponentStateEndpointMerger.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ComponentStateEndpointMerger.java @@ -19,6 +19,7 @@ import java.net.URI; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; @@ -27,6 +28,7 @@ import org.apache.nifi.cluster.manager.NodeResponse; import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.components.state.Scope; import org.apache.nifi.controller.state.SortedStateUtils; import org.apache.nifi.web.api.dto.ComponentStateDTO; import org.apache.nifi.web.api.dto.StateEntryDTO; @@ -60,41 +62,91 @@ protected ComponentStateDTO getDto(ComponentStateEntity entity) { } @Override - protected void mergeResponses(ComponentStateDTO clientDto, Map dtoMap, Set successfulResponses, Set problematicResponses) { - List localStateEntries = new ArrayList<>(); + public void mergeResponses(ComponentStateDTO clientDto, Map dtoMap, + Set successfulResponses, Set problematicResponses) { - int totalStateEntries = 0; + // If there're more than 1 node returning external state, then it's a per node external state. + final boolean externalPerNode = dtoMap.values().stream() + .filter(component -> (component.getExternalState() != null && component.getExternalState().getState() != null)) + .count() > 1; + + final StateMapDTOMerger[] mergers = { + new StateMapDTOMerger(Scope.LOCAL, node -> node.getLocalState(), (merged, state) -> merged.setLocalState(state), true), + new StateMapDTOMerger(Scope.CLUSTER, node -> node.getClusterState(), (merged, state) -> merged.setClusterState(state), false), + new StateMapDTOMerger(Scope.EXTERNAL, node -> node.getExternalState(), (merged, state) -> merged.setExternalState(state), externalPerNode) + }; + + // Loop through nodes for (final Map.Entry nodeEntry : dtoMap.entrySet()) { final ComponentStateDTO nodeComponentState = nodeEntry.getValue(); final NodeIdentifier nodeId = nodeEntry.getKey(); final String nodeAddress = nodeId.getApiAddress() + ":" + nodeId.getApiPort(); - final StateMapDTO nodeLocalStateMap = nodeComponentState.getLocalState(); - if (nodeLocalStateMap.getState() != null) { - totalStateEntries += nodeLocalStateMap.getTotalEntryCount(); - - for (final StateEntryDTO nodeStateEntry : nodeLocalStateMap.getState()) { - if (nodeStateEntry.getClusterNodeId() == null || nodeStateEntry.getClusterNodeAddress() == null) { - nodeStateEntry.setClusterNodeId(nodeId.getId()); - nodeStateEntry.setClusterNodeAddress(nodeAddress); + for (final StateMapDTOMerger merger : mergers) { + final StateMapDTO nodeStateMapDTO = merger.stateGetter.apply(nodeComponentState); + if (nodeStateMapDTO != null && nodeStateMapDTO.getState() != null) { + final StateMapDTO mergedStateMapDTO = merger.getMergedStateMapDTO(); + final List stateEntries = mergedStateMapDTO.getState(); + mergedStateMapDTO.setTotalEntryCount(mergedStateMapDTO.getTotalEntryCount() + nodeStateMapDTO.getTotalEntryCount()); + + for (final StateEntryDTO nodeStateEntry : nodeStateMapDTO.getState()) { + if (merger.perNode + && (nodeStateEntry.getClusterNodeId() == null || nodeStateEntry.getClusterNodeAddress() == null)) { + nodeStateEntry.setClusterNodeId(nodeId.getId()); + nodeStateEntry.setClusterNodeAddress(nodeAddress); + } + + stateEntries.add(nodeStateEntry); } - - localStateEntries.add(nodeStateEntry); } } } - // ensure appropriate sort - Collections.sort(localStateEntries, SortedStateUtils.getEntryDtoComparator()); + Arrays.stream(mergers).filter(m -> m.mergedStateMapDTO != null).forEach(m -> { + // ensure appropriate sort + final List stateEntries = m.mergedStateMapDTO.getState(); + Collections.sort(stateEntries, SortedStateUtils.getEntryDtoComparator()); + + // sublist if necessary + if (stateEntries.size() > SortedStateUtils.MAX_COMPONENT_STATE_ENTRIES) { + m.mergedStateMapDTO.setState(stateEntries.subList(0, SortedStateUtils.MAX_COMPONENT_STATE_ENTRIES)); + } + + m.stateSetter.apply(clientDto, m.mergedStateMapDTO); + }); - // sublist if necessary - if (localStateEntries.size() > SortedStateUtils.MAX_COMPONENT_STATE_ENTRIES) { - localStateEntries = localStateEntries.subList(0, SortedStateUtils.MAX_COMPONENT_STATE_ENTRIES); + } + + private interface StateGetter { + StateMapDTO apply(ComponentStateDTO node); + } + private interface StateSetter { + void apply(ComponentStateDTO merged, StateMapDTO state); + } + + private static class StateMapDTOMerger { + private final StateGetter stateGetter; + private final StateSetter stateSetter; + private final Scope scope; + private final boolean perNode; + private StateMapDTO mergedStateMapDTO; + + public StateMapDTOMerger(final Scope scope, final StateGetter stateGetter, final StateSetter stateSetter, final boolean perNode) { + this.scope = scope; + this.stateGetter = stateGetter; + this.stateSetter = stateSetter; + this.perNode = perNode; } - // add all the local state entries - clientDto.getLocalState().setTotalEntryCount(totalStateEntries); - clientDto.getLocalState().setState(localStateEntries); + public StateMapDTO getMergedStateMapDTO() { + if (mergedStateMapDTO == null) { + mergedStateMapDTO = new StateMapDTO(); + mergedStateMapDTO.setScope(scope.toString()); + mergedStateMapDTO.setTotalEntryCount(0); + mergedStateMapDTO.setState(new ArrayList<>()); + } + return mergedStateMapDTO; + } } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/endpoints/TestComponentStateEndpointMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/endpoints/TestComponentStateEndpointMerger.java new file mode 100644 index 000000000000..77b71343db36 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/endpoints/TestComponentStateEndpointMerger.java @@ -0,0 +1,437 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.cluster.coordination.http.endpoints; + +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.web.api.dto.ComponentStateDTO; +import org.apache.nifi.web.api.dto.StateEntryDTO; +import org.apache.nifi.web.api.dto.StateMapDTO; +import org.codehaus.jackson.map.ObjectMapper; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.StringWriter; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +public class TestComponentStateEndpointMerger { + + private static final Logger logger = LoggerFactory.getLogger(TestComponentStateEndpointMerger.class); + + private static final NodeIdentifier NODE_1 = new NodeIdentifier("1", "localhost", 9101, "localhost", 9102, "localhost", 9103, 9104, false); + private static final NodeIdentifier NODE_2 = new NodeIdentifier("2", "localhost", 9201, "localhost", 9202, "localhost", 9203, 9204, false); + private static final NodeIdentifier NODE_3 = new NodeIdentifier("3", "localhost", 9301, "localhost", 9302, "localhost", 9303, 9304, false); + + private static class StateMapDTOBuilder { + private final StateMapDTO stateMap; + private final List stateEntries; + + public StateMapDTOBuilder(Scope scope) { + stateEntries = new ArrayList<>(); + stateMap = new StateMapDTO(); + stateMap.setScope(scope.toString()); + stateMap.setState(stateEntries); + } + + public StateMapDTOBuilder addState(String key, String value) { + final StateEntryDTO stateEntryDTO = new StateEntryDTO(); + stateEntryDTO.setKey(key); + stateEntryDTO.setValue(value); + stateEntries.add(stateEntryDTO); + stateMap.setTotalEntryCount(stateEntries.size()); + return this; + } + + } + + private static class ComponentStateDTOBuilder { + private final ComponentStateDTO state; + + public ComponentStateDTOBuilder() { + this.state = new ComponentStateDTO(); + } + + public StateMapDTOBuilder buildState(Scope scope) { + final StateMapDTOBuilder builder = new StateMapDTOBuilder(scope); + switch (scope) { + case LOCAL: + state.setLocalState(builder.stateMap); + break; + case CLUSTER: + state.setClusterState(builder.stateMap); + break; + case EXTERNAL: + state.setExternalState(builder.stateMap); + break; + } + return builder; + } + + } + + private void printDTO(ComponentStateDTO clientDto) throws IOException { + try (final StringWriter writer = new StringWriter();) { + new ObjectMapper().writeValue(writer, clientDto); + logger.info("clientDto={}", writer); + } + } + + private ComponentStateEndpointMerger merger = new ComponentStateEndpointMerger(); + + private ComponentStateDTOBuilder builder1; + private ComponentStateDTOBuilder builder2; + private ComponentStateDTOBuilder builder3; + private Map dtoMap; + private ComponentStateDTO clientDto; + + @Before + public void before() { + builder1 = new ComponentStateDTOBuilder(); + builder2 = new ComponentStateDTOBuilder(); + builder3 = new ComponentStateDTOBuilder(); + dtoMap = new HashMap<>(); + clientDto = builder1.state; + dtoMap.put(NODE_1, builder1.state); + dtoMap.put(NODE_2, builder2.state); + dtoMap.put(NODE_3, builder3.state); + + // Local scope state is always set. + builder1.buildState(Scope.LOCAL); + builder2.buildState(Scope.LOCAL); + builder3.buildState(Scope.LOCAL); + + } + + @Test + public void testMergeNothing() throws Exception { + + merger.mergeResponses(clientDto, dtoMap, null, null); + + printDTO(clientDto); + + assertNotNull(clientDto.getLocalState()); + assertEquals(0, clientDto.getLocalState().getTotalEntryCount()); + assertEquals(0, clientDto.getLocalState().getState().size()); + assertNull(clientDto.getClusterState()); + assertNull(clientDto.getExternalState()); + } + + @Test + public void testMergeLocal() throws Exception { + + builder1.buildState(Scope.LOCAL) + .addState("k1-1", "v1-1"); + builder2.buildState(Scope.LOCAL) + .addState("k2-1", "v2-1") + .addState("k2-2", "v2-2"); + builder3.buildState(Scope.LOCAL) + .addState("k3-1", "v3-1") + .addState("k3-2", "v3-2") + .addState("k3-3", "v3-3"); + + merger.mergeResponses(clientDto, dtoMap, null, null); + + printDTO(clientDto); + + // Local + final StateMapDTO localStateMap = clientDto.getLocalState(); + assertNotNull(localStateMap); + assertEquals(6, localStateMap.getTotalEntryCount()); + final List localState = localStateMap.getState(); + assertEquals(6, localState.size()); + assertEquals("k1-1", localState.get(0).getKey()); + assertEquals("v1-1", localState.get(0).getValue()); + assertEquals("k2-1", localState.get(1).getKey()); + assertEquals("v2-1", localState.get(1).getValue()); + assertEquals("k2-2", localState.get(2).getKey()); + assertEquals("v2-2", localState.get(2).getValue()); + assertEquals("k3-1", localState.get(3).getKey()); + assertEquals("v3-1", localState.get(3).getValue()); + assertEquals("k3-2", localState.get(4).getKey()); + assertEquals("v3-2", localState.get(4).getValue()); + assertEquals("k3-3", localState.get(5).getKey()); + assertEquals("v3-3", localState.get(5).getValue()); + + assertEquals("1", localState.get(0).getClusterNodeId()); + assertEquals("2", localState.get(1).getClusterNodeId()); + assertEquals("2", localState.get(2).getClusterNodeId()); + assertEquals("3", localState.get(3).getClusterNodeId()); + assertEquals("3", localState.get(4).getClusterNodeId()); + assertEquals("3", localState.get(5).getClusterNodeId()); + + assertEquals("localhost:9101", localState.get(0).getClusterNodeAddress()); + assertEquals("localhost:9201", localState.get(1).getClusterNodeAddress()); + assertEquals("localhost:9201", localState.get(2).getClusterNodeAddress()); + assertEquals("localhost:9301", localState.get(3).getClusterNodeAddress()); + assertEquals("localhost:9301", localState.get(4).getClusterNodeAddress()); + assertEquals("localhost:9301", localState.get(5).getClusterNodeAddress()); + + // Cluster + assertNull(clientDto.getClusterState()); + + // External + assertNull(clientDto.getExternalState()); + } + + @Test + public void testMergeCluster() throws Exception { + + // Node2 is the primary node. + builder2.buildState(Scope.CLUSTER) + .addState("c-k1", "c-v1") + .addState("c-k2", "c-v2"); + + merger.mergeResponses(clientDto, dtoMap, null, null); + + printDTO(clientDto); + + // Local + assertNotNull(clientDto.getLocalState()); + assertEquals(0, clientDto.getLocalState().getTotalEntryCount()); + assertEquals(0, clientDto.getLocalState().getState().size()); + + // Cluster + final StateMapDTO clusterStateMap = clientDto.getClusterState(); + assertNotNull(clusterStateMap); + assertEquals(2, clusterStateMap.getTotalEntryCount()); + final List clusterState = clusterStateMap.getState(); + assertEquals(2, clusterState.size()); + + assertEquals("c-k1", clusterState.get(0).getKey()); + assertEquals("c-v1", clusterState.get(0).getValue()); + assertEquals("c-k2", clusterState.get(1).getKey()); + assertEquals("c-v2", clusterState.get(1).getValue()); + + assertNull(clusterState.get(0).getClusterNodeId()); + assertNull(clusterState.get(1).getClusterNodeId()); + assertNull(clusterState.get(0).getClusterNodeAddress()); + assertNull(clusterState.get(1).getClusterNodeAddress()); + + // External + assertNull(clientDto.getExternalState()); + + } + + @Test + public void testMergeExternal() throws Exception { + + // Node2 is the primary node. + builder2.buildState(Scope.EXTERNAL) + .addState("e-k1", "e-v1") + .addState("e-k2", "e-v2"); + + merger.mergeResponses(clientDto, dtoMap, null, null); + + printDTO(clientDto); + + // Local + assertNotNull(clientDto.getLocalState()); + assertEquals(0, clientDto.getLocalState().getTotalEntryCount()); + assertEquals(0, clientDto.getLocalState().getState().size()); + + // Cluster + assertNull(clientDto.getClusterState()); + + // External + final StateMapDTO externalStateMap = clientDto.getExternalState(); + assertNotNull(externalStateMap); + assertEquals(2, externalStateMap.getTotalEntryCount()); + final List externalState = externalStateMap.getState(); + assertEquals(2, externalState.size()); + + assertEquals("e-k1", externalState.get(0).getKey()); + assertEquals("e-v1", externalState.get(0).getValue()); + assertEquals("e-k2", externalState.get(1).getKey()); + assertEquals("e-v2", externalState.get(1).getValue()); + + assertNull(externalState.get(0).getClusterNodeId()); + assertNull(externalState.get(1).getClusterNodeId()); + assertNull(externalState.get(0).getClusterNodeAddress()); + assertNull(externalState.get(1).getClusterNodeAddress()); + + } + + @Test + public void testMergeExternalPerNode() throws Exception { + + builder1.buildState(Scope.EXTERNAL) + .addState("e-k11", "e-v11") + .addState("e-k12", "e-v12"); + + builder2.buildState(Scope.EXTERNAL) + .addState("e-k21", "e-v21") + .addState("e-k22", "e-v22"); + + builder3.buildState(Scope.EXTERNAL) + .addState("e-k31", "e-v31") + .addState("e-k32", "e-v32"); + + + merger.mergeResponses(clientDto, dtoMap, null, null); + + printDTO(clientDto); + + // Local + assertNotNull(clientDto.getLocalState()); + assertEquals(0, clientDto.getLocalState().getTotalEntryCount()); + assertEquals(0, clientDto.getLocalState().getState().size()); + + // Cluster + assertNull(clientDto.getClusterState()); + + // External + final StateMapDTO externalStateMap = clientDto.getExternalState(); + assertNotNull(externalStateMap); + assertEquals(6, externalStateMap.getTotalEntryCount()); + final List externalState = externalStateMap.getState(); + assertEquals(6, externalState.size()); + + assertEquals("e-k11", externalState.get(0).getKey()); + assertEquals("e-v11", externalState.get(0).getValue()); + assertEquals("e-k12", externalState.get(1).getKey()); + assertEquals("e-v12", externalState.get(1).getValue()); + + assertEquals("e-k21", externalState.get(2).getKey()); + assertEquals("e-v21", externalState.get(2).getValue()); + assertEquals("e-k22", externalState.get(3).getKey()); + assertEquals("e-v22", externalState.get(3).getValue()); + + assertEquals("e-k31", externalState.get(4).getKey()); + assertEquals("e-v31", externalState.get(4).getValue()); + assertEquals("e-k32", externalState.get(5).getKey()); + assertEquals("e-v32", externalState.get(5).getValue()); + + assertEquals("1", externalState.get(0).getClusterNodeId()); + assertEquals("1", externalState.get(1).getClusterNodeId()); + assertEquals("2", externalState.get(2).getClusterNodeId()); + assertEquals("2", externalState.get(3).getClusterNodeId()); + assertEquals("3", externalState.get(4).getClusterNodeId()); + assertEquals("3", externalState.get(5).getClusterNodeId()); + assertEquals("localhost:9101", externalState.get(0).getClusterNodeAddress()); + assertEquals("localhost:9101", externalState.get(1).getClusterNodeAddress()); + assertEquals("localhost:9201", externalState.get(2).getClusterNodeAddress()); + assertEquals("localhost:9201", externalState.get(3).getClusterNodeAddress()); + assertEquals("localhost:9301", externalState.get(4).getClusterNodeAddress()); + assertEquals("localhost:9301", externalState.get(5).getClusterNodeAddress()); + + } + + @Test + public void testMergeAll() throws Exception { + + builder1.buildState(Scope.LOCAL) + .addState("k1-1", "v1-1"); + builder2.buildState(Scope.LOCAL) + .addState("k2-1", "v2-1") + .addState("k2-2", "v2-2"); + builder3.buildState(Scope.LOCAL) + .addState("k3-1", "v3-1") + .addState("k3-2", "v3-2") + .addState("k3-3", "v3-3"); + + // Node2 is the primary node. + builder2.buildState(Scope.CLUSTER) + .addState("c-k1", "c-v1") + .addState("c-k2", "c-v2"); + builder2.buildState(Scope.EXTERNAL) + .addState("e-k1", "e-v1") + .addState("e-k2", "e-v2"); + + merger.mergeResponses(clientDto, dtoMap, null, null); + + printDTO(clientDto); + + // Local + final StateMapDTO localStateMap = clientDto.getLocalState(); + assertNotNull(localStateMap); + assertEquals(6, localStateMap.getTotalEntryCount()); + final List localState = localStateMap.getState(); + assertEquals(6, localState.size()); + assertEquals("k1-1", localState.get(0).getKey()); + assertEquals("v1-1", localState.get(0).getValue()); + assertEquals("k2-1", localState.get(1).getKey()); + assertEquals("v2-1", localState.get(1).getValue()); + assertEquals("k2-2", localState.get(2).getKey()); + assertEquals("v2-2", localState.get(2).getValue()); + assertEquals("k3-1", localState.get(3).getKey()); + assertEquals("v3-1", localState.get(3).getValue()); + assertEquals("k3-2", localState.get(4).getKey()); + assertEquals("v3-2", localState.get(4).getValue()); + assertEquals("k3-3", localState.get(5).getKey()); + assertEquals("v3-3", localState.get(5).getValue()); + + assertEquals("1", localState.get(0).getClusterNodeId()); + assertEquals("2", localState.get(1).getClusterNodeId()); + assertEquals("2", localState.get(2).getClusterNodeId()); + assertEquals("3", localState.get(3).getClusterNodeId()); + assertEquals("3", localState.get(4).getClusterNodeId()); + assertEquals("3", localState.get(5).getClusterNodeId()); + + assertEquals("localhost:9101", localState.get(0).getClusterNodeAddress()); + assertEquals("localhost:9201", localState.get(1).getClusterNodeAddress()); + assertEquals("localhost:9201", localState.get(2).getClusterNodeAddress()); + assertEquals("localhost:9301", localState.get(3).getClusterNodeAddress()); + assertEquals("localhost:9301", localState.get(4).getClusterNodeAddress()); + assertEquals("localhost:9301", localState.get(5).getClusterNodeAddress()); + + // Cluster + final StateMapDTO clusterStateMap = clientDto.getClusterState(); + assertNotNull(clusterStateMap); + assertEquals(2, clusterStateMap.getTotalEntryCount()); + final List clusterState = clusterStateMap.getState(); + assertEquals(2, clusterState.size()); + + assertEquals("c-k1", clusterState.get(0).getKey()); + assertEquals("c-v1", clusterState.get(0).getValue()); + assertEquals("c-k2", clusterState.get(1).getKey()); + assertEquals("c-v2", clusterState.get(1).getValue()); + + assertNull(clusterState.get(0).getClusterNodeId()); + assertNull(clusterState.get(1).getClusterNodeId()); + assertNull(clusterState.get(0).getClusterNodeAddress()); + assertNull(clusterState.get(1).getClusterNodeAddress()); + + // External + final StateMapDTO externalStateMap = clientDto.getExternalState(); + assertNotNull(externalStateMap); + assertEquals(2, externalStateMap.getTotalEntryCount()); + final List externalState = externalStateMap.getState(); + assertEquals(2, externalState.size()); + + assertEquals("e-k1", externalState.get(0).getKey()); + assertEquals("e-v1", externalState.get(0).getValue()); + assertEquals("e-k2", externalState.get(1).getKey()); + assertEquals("e-v2", externalState.get(1).getValue()); + + assertNull(externalState.get(0).getClusterNodeId()); + assertNull(externalState.get(1).getClusterNodeId()); + assertNull(externalState.get(0).getClusterNodeAddress()); + assertNull(externalState.get(1).getClusterNodeAddress()); + + } +} 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 index 639f8a25bbdd..5092b8713123 100644 --- 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 @@ -41,6 +41,10 @@ public StandardStateManager(final StateProvider localProvider, final StateProvid } private StateProvider getProvider(final Scope scope) { + if (scope == Scope.EXTERNAL) { + throw new IllegalArgumentException("EXTERNAL scope is not managed by StateManager." + + " A component should implement ExternalStateManager by itself to support EXTERNAL scope state."); + } if (scope == Scope.LOCAL || clusterProvider == null || !clusterProvider.isEnabled()) { return localProvider; } 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 index 2e7f5c948427..08542098e65b 100644 --- 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 @@ -23,6 +23,7 @@ import java.util.HashMap; import java.util.Map; +import org.apache.nifi.components.state.StandardStateMap; import org.apache.nifi.components.state.StateMap; import org.wali.SerDe; import org.wali.UpdateType; 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 index fc691fb1813d..c2fcfbd12f86 100644 --- 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 @@ -37,7 +37,7 @@ 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.components.state.StandardStateMap; import org.apache.nifi.controller.state.StateMapSerDe; import org.apache.nifi.controller.state.StateMapUpdate; import org.apache.nifi.controller.state.providers.AbstractStateProvider; 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 index ba1aa30170a0..131db7de8ac6 100644 --- 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 @@ -39,7 +39,7 @@ import org.apache.nifi.components.state.StateMap; import org.apache.nifi.components.state.StateProviderInitializationContext; import org.apache.nifi.components.state.exception.StateTooLargeException; -import org.apache.nifi.controller.state.StandardStateMap; +import org.apache.nifi.components.state.StandardStateMap; import org.apache.nifi.controller.state.providers.AbstractStateProvider; import org.apache.nifi.processor.util.StandardValidators; import org.apache.zookeeper.CreateMode; 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 index 63bfbade47bb..db8c4c592131 100644 --- 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 @@ -28,6 +28,7 @@ import java.util.HashMap; import java.util.Map; +import org.apache.nifi.components.state.StandardStateMap; import org.apache.nifi.components.state.StateMap; import org.junit.Test; import org.wali.UpdateType; 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 08a56e249ed4..2b6ba6f74eb6 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 @@ -71,6 +71,7 @@ import org.apache.nifi.web.api.dto.status.RemoteProcessGroupStatusDTO; import org.apache.nifi.web.api.dto.status.StatusHistoryDTO; import org.apache.nifi.web.api.entity.AccessPolicyEntity; +import org.apache.nifi.web.api.entity.ClearComponentStateResultEntity; import org.apache.nifi.web.api.entity.ConnectionEntity; import org.apache.nifi.web.api.entity.ControllerBulletinsEntity; import org.apache.nifi.web.api.entity.ControllerConfigurationEntity; @@ -1092,8 +1093,9 @@ public interface NiFiServiceFacade { * Clears the state for the specified processor. * * @param processorId the processor id + * @return the result of clear operation */ - void clearProcessorState(String processorId); + ClearComponentStateResultEntity clearProcessorState(String processorId); /** * Gets the state for the specified controller service. @@ -1114,8 +1116,9 @@ public interface NiFiServiceFacade { * Clears the state for the specified controller service. * * @param controllerServiceId the controller service id + * @return the result of clear operation */ - void clearControllerServiceState(String controllerServiceId); + ClearComponentStateResultEntity clearControllerServiceState(String controllerServiceId); /** * Gets the state for the specified reporting task. @@ -1136,8 +1139,9 @@ public interface NiFiServiceFacade { * Clears the state for the specified reporting task. * * @param reportingTaskId the reporting task id + * @return the result of clear operation */ - void clearReportingTaskState(String reportingTaskId); + ClearComponentStateResultEntity clearReportingTaskState(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/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 3b010d529b33..46bc9a16335a 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 @@ -48,9 +48,11 @@ import org.apache.nifi.cluster.event.NodeEvent; import org.apache.nifi.cluster.manager.exception.UnknownNodeException; import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.components.ConfigurableComponent; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationResult; import org.apache.nifi.components.Validator; +import org.apache.nifi.components.state.ExternalStateManager; import org.apache.nifi.components.state.Scope; import org.apache.nifi.components.state.StateMap; import org.apache.nifi.connectable.Connection; @@ -146,6 +148,7 @@ import org.apache.nifi.web.api.dto.status.StatusHistoryDTO; import org.apache.nifi.web.api.entity.AccessPolicyEntity; import org.apache.nifi.web.api.entity.AccessPolicySummaryEntity; +import org.apache.nifi.web.api.entity.ClearComponentStateResultEntity; import org.apache.nifi.web.api.entity.ConnectionEntity; import org.apache.nifi.web.api.entity.ControllerBulletinsEntity; import org.apache.nifi.web.api.entity.ControllerConfigurationEntity; @@ -172,6 +175,7 @@ import org.apache.nifi.web.api.entity.UserGroupEntity; import org.apache.nifi.web.controller.ControllerFacade; import org.apache.nifi.web.dao.AccessPolicyDAO; +import org.apache.nifi.web.dao.ComponentStateDAO; import org.apache.nifi.web.dao.ConnectionDAO; import org.apache.nifi.web.dao.ControllerServiceDAO; import org.apache.nifi.web.dao.FunnelDAO; @@ -250,6 +254,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { private AccessPolicyDAO accessPolicyDAO; private ClusterCoordinator clusterCoordinator; private HeartbeatMonitor heartbeatMonitor; + private ComponentStateDAO componentStateDAO; // administrative services private AuditService auditService; @@ -815,14 +820,44 @@ public CounterDTO updateCounter(final String counterId) { return dtoFactory.createCounterDto(controllerFacade.resetCounter(counterId)); } + private ClearComponentStateResultEntity clearComponentState(final ConfigurableComponent component) { + + logger.debug("clearComponentState: component={}, shouldManageClusterState={}, shouldManageExternalState={}", + component, shouldManageClusterState(), shouldManageExternalState(component)); + + final ClearComponentStateResultEntity entity = new ClearComponentStateResultEntity(); + componentStateDAO.clearState(component, Scope.LOCAL); + + try { + if (shouldManageClusterState()) { + componentStateDAO.clearState(component, Scope.CLUSTER); + } + + if (shouldManageExternalState(component)) { + componentStateDAO.clearState(component, Scope.EXTERNAL); + } + } catch (Exception e) { + // If an exception is thrown for these cluster-wide or external state, + // the cause would be related to the underlying storage system with high possibility. + // If we throw exception here, cluster coordinator will disconnects this node from a cluster. + // See: NodeClusterCoordinator.afterRequest() + // Removing node doesn't help since it's related to external system, and other node will have the same issue. + // So instead, return entity with message. + logger.warn("Failed to clear state for {} due to {}", component, e, e); + entity.setCleared(false); + entity.setMessage(e.getMessage()); + } + return entity; + } + @Override public void verifyCanClearProcessorState(final String processorId) { processorDAO.verifyClearState(processorId); } @Override - public void clearProcessorState(final String processorId) { - processorDAO.clearState(processorId); + public ClearComponentStateResultEntity clearProcessorState(final String processorId) { + return clearComponentState(processorDAO.getProcessor(processorId).getProcessor()); } @Override @@ -831,8 +866,8 @@ public void verifyCanClearControllerServiceState(final String controllerServiceI } @Override - public void clearControllerServiceState(final String controllerServiceId) { - controllerServiceDAO.clearState(controllerServiceId); + public ClearComponentStateResultEntity clearControllerServiceState(final String controllerServiceId) { + return clearComponentState(controllerServiceDAO.getControllerService(controllerServiceId).getControllerServiceImplementation()); } @Override @@ -841,8 +876,8 @@ public void verifyCanClearReportingTaskState(final String reportingTaskId) { } @Override - public void clearReportingTaskState(final String reportingTaskId) { - reportingTaskDAO.clearState(reportingTaskId); + public ClearComponentStateResultEntity clearReportingTaskState(final String reportingTaskId) { + return clearComponentState(reportingTaskDAO.getReportingTask(reportingTaskId).getReportingTask()); } @Override @@ -1902,34 +1937,50 @@ public ControllerStatusDTO getControllerStatus() { return controllerFacade.getControllerStatus(); } + private boolean shouldManageClusterState() { + return !isClustered() || controllerFacade.isPrimary(); + } + + private boolean shouldManageExternalState(final ConfigurableComponent component) { + return (component instanceof ExternalStateManager) + && (!isClustered() + || ExternalStateManager.ExternalStateScope.CLUSTER != ((ExternalStateManager)component).getExternalStateScope() + || controllerFacade.isPrimary()); + } + + private ComponentStateDTO getComponentState(final ConfigurableComponent component) { + + logger.debug("getComponentState: component={}, shouldManageClusterState={}, shouldManageExternalState={}", + component, shouldManageClusterState(), shouldManageExternalState(component)); + + try { + final StateMap clusterState = shouldManageClusterState() ? componentStateDAO.getState(component, Scope.CLUSTER) : null; + logger.debug("clusterState={}", clusterState); + final StateMap localState = componentStateDAO.getState(component, Scope.LOCAL); + logger.debug("localState={}", localState); + final StateMap externalState = shouldManageExternalState(component) ? componentStateDAO.getState(component, Scope.EXTERNAL) : null; + logger.debug("externalState={}", externalState); + + return dtoFactory.createComponentStateDTO(component, localState, clusterState, externalState); + } catch (Exception e) { + logger.error("Failed to retrieve state for {} due to {}", component, e, e); + throw e; + } + } + @Override public ComponentStateDTO getProcessorState(final String processorId) { - final StateMap clusterState = isClustered() ? processorDAO.getState(processorId, Scope.CLUSTER) : null; - final StateMap localState = processorDAO.getState(processorId, Scope.LOCAL); - - // processor will be non null as it was already found when getting the state - final ProcessorNode processor = processorDAO.getProcessor(processorId); - return dtoFactory.createComponentStateDTO(processorId, processor.getProcessor().getClass(), localState, clusterState); + return getComponentState(processorDAO.getProcessor(processorId).getProcessor()); } @Override public ComponentStateDTO getControllerServiceState(final 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); + return getComponentState(controllerServiceDAO.getControllerService(controllerServiceId).getControllerServiceImplementation()); } @Override public ComponentStateDTO getReportingTaskState(final 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); + return getComponentState(reportingTaskDAO.getReportingTask(reportingTaskId).getReportingTask()); } @Override @@ -3008,4 +3059,8 @@ public void setHeartbeatMonitor(final HeartbeatMonitor heartbeatMonitor) { public void setBulletinRepository(final BulletinRepository bulletinRepository) { this.bulletinRepository = bulletinRepository; } + + 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/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 4eb1d1ae2e91..ab49f2451d83 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 @@ -38,6 +38,7 @@ import org.apache.nifi.web.api.dto.ControllerServiceDTO; import org.apache.nifi.web.api.dto.PropertyDescriptorDTO; import org.apache.nifi.web.api.dto.RevisionDTO; +import org.apache.nifi.web.api.entity.ClearComponentStateResultEntity; import org.apache.nifi.web.api.entity.ComponentStateEntity; import org.apache.nifi.web.api.entity.ControllerServiceEntity; import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentsEntity; @@ -319,7 +320,7 @@ public Response getState( * * @param httpServletRequest servlet request * @param id The id of the controller service - * @return a componentStateEntity + * @return a clearComponentStateEntity */ @POST @Consumes(MediaType.WILDCARD) @@ -328,7 +329,7 @@ public Response getState( // TODO - @PreAuthorize("hasAnyRole('ROLE_DFM')") @ApiOperation( value = "Clears the state for a controller service", - response = ComponentStateDTO.class, + response = ClearComponentStateResultEntity.class, authorizations = { @Authorization(value = "Data Flow Manager", type = "ROLE_DFM") } @@ -367,11 +368,7 @@ public Response clearState( return generateContinueResponse().build(); } - // get the component state - serviceFacade.clearControllerServiceState(id); - - // generate the response entity - final ComponentStateEntity entity = new ComponentStateEntity(); + ClearComponentStateResultEntity entity = serviceFacade.clearControllerServiceState(id); // generate the response return clusterContext(generateOkResponse(entity)).build(); 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 bf104f5034a2..2ea9f146fb29 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 @@ -36,6 +36,7 @@ import org.apache.nifi.web.api.dto.ProcessorConfigDTO; import org.apache.nifi.web.api.dto.ProcessorDTO; import org.apache.nifi.web.api.dto.PropertyDescriptorDTO; +import org.apache.nifi.web.api.entity.ClearComponentStateResultEntity; import org.apache.nifi.web.api.entity.ComponentStateEntity; import org.apache.nifi.web.api.entity.ProcessorEntity; import org.apache.nifi.web.api.entity.PropertyDescriptorEntity; @@ -276,7 +277,7 @@ public Response getPropertyDescriptor( // TODO - @PreAuthorize("hasAnyRole('ROLE_DFM')") @ApiOperation( value = "Gets the state for a processor", - response = ComponentStateDTO.class, + response = ComponentStateEntity.class, authorizations = { @Authorization(value = "Data Flow Manager", type = "ROLE_DFM") } @@ -323,7 +324,7 @@ public Response getState( * * @param httpServletRequest servlet request * @param id The id of the processor - * @return a componentStateEntity + * @return a clearComponentStateEntity * @throws InterruptedException if interrupted */ @POST @@ -333,7 +334,7 @@ public Response getState( // TODO - @PreAuthorize("hasAnyRole('ROLE_DFM')") @ApiOperation( value = "Clears the state for a processor", - response = ComponentStateDTO.class, + response = ClearComponentStateResultEntity.class, authorizations = { @Authorization(value = "Data Flow Manager", type = "ROLE_DFM") } @@ -372,11 +373,7 @@ public Response clearState( return generateContinueResponse().build(); } - // get the component state - serviceFacade.clearProcessorState(id); - - // generate the response entity - final ComponentStateEntity entity = new ComponentStateEntity(); + final ClearComponentStateResultEntity entity = serviceFacade.clearProcessorState(id); // generate the response return clusterContext(generateOkResponse(entity)).build(); 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 1025de2bf7ef..d7fc7415c881 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 @@ -35,6 +35,7 @@ import org.apache.nifi.web.api.dto.ComponentStateDTO; import org.apache.nifi.web.api.dto.PropertyDescriptorDTO; import org.apache.nifi.web.api.dto.ReportingTaskDTO; +import org.apache.nifi.web.api.entity.ClearComponentStateResultEntity; import org.apache.nifi.web.api.entity.ComponentStateEntity; import org.apache.nifi.web.api.entity.PropertyDescriptorEntity; import org.apache.nifi.web.api.entity.ReportingTaskEntity; @@ -306,7 +307,7 @@ public Response getState( * * @param httpServletRequest servlet request * @param id The id of the reporting task - * @return a componentStateEntity + * @return a clearComponentStateEntity */ @POST @Consumes(MediaType.WILDCARD) @@ -315,7 +316,7 @@ public Response getState( // TODO - @PreAuthorize("hasAnyRole('ROLE_DFM')") @ApiOperation( value = "Clears the state for a reporting task", - response = ComponentStateDTO.class, + response = ClearComponentStateResultEntity.class, authorizations = { @Authorization(value = "Data Flow Manager", type = "ROLE_DFM") } @@ -354,11 +355,7 @@ public Response clearState( return generateContinueResponse().build(); } - // get the component state - serviceFacade.clearReportingTaskState(id); - - // generate the response entity - final ComponentStateEntity entity = new ComponentStateEntity(); + final ClearComponentStateResultEntity entity = serviceFacade.clearReportingTaskState(id); // generate the response return clusterContext(generateOkResponse(entity)).build(); 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 93afccf2c51a..452db615d4dc 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 @@ -49,6 +49,7 @@ import org.apache.nifi.cluster.manager.StatusMerger; import org.apache.nifi.cluster.protocol.NodeIdentifier; import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.ConfigurableComponent; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationResult; import org.apache.nifi.components.state.Scope; @@ -313,17 +314,21 @@ public HistoryDTO createHistoryDto(final History history) { /** * Creates a ComponentStateDTO for the given component and state's. * - * @param componentId component id + * @param component component * @param localState local state * @param clusterState cluster state + * @param externalState external state * @return dto */ - public ComponentStateDTO createComponentStateDTO(final String componentId, final Class componentClass, final StateMap localState, final StateMap clusterState) { + public ComponentStateDTO createComponentStateDTO(final ConfigurableComponent component, final StateMap localState, + final StateMap clusterState, final StateMap externalState) { final ComponentStateDTO dto = new ComponentStateDTO(); - dto.setComponentId(componentId); - dto.setStateDescription(getStateDescription(componentClass)); + dto.setComponentId(component.getIdentifier()); + dto.setStateDescription(getStateDescription(component.getClass())); dto.setLocalState(createStateMapDTO(Scope.LOCAL, localState)); dto.setClusterState(createStateMapDTO(Scope.CLUSTER, clusterState)); + dto.setExternalState(createStateMapDTO(Scope.EXTERNAL, externalState)); + return dto; } 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 2152c7696945..afc2659f47a8 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 @@ -41,6 +41,7 @@ import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.Counter; import org.apache.nifi.controller.FlowController; +import org.apache.nifi.controller.NodeTypeProvider; import org.apache.nifi.controller.ProcessorNode; import org.apache.nifi.controller.ReportingTaskNode; import org.apache.nifi.controller.ScheduledState; @@ -134,7 +135,7 @@ import static org.apache.nifi.controller.FlowController.ROOT_GROUP_ID_ALIAS; -public class ControllerFacade implements Authorizable { +public class ControllerFacade implements Authorizable, NodeTypeProvider { private static final Logger logger = LoggerFactory.getLogger(ControllerFacade.class); @@ -367,13 +368,16 @@ public NodeIdentifier getNodeId() { return flowController.getNodeId(); } - /** - * @return true if is clustered - */ + @Override public boolean isClustered() { return flowController.isClustered(); } + @Override + public boolean isPrimary() { + return flowController.isPrimary(); + } + /** * Gets the name of this controller. * 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 index 636addb02fa8..b244e04c05e2 100644 --- 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 @@ -16,59 +16,27 @@ */ package org.apache.nifi.web.dao; +import org.apache.nifi.components.ConfigurableComponent; 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. + * Gets the state for the specified component. * - * @param processor processor + * @param component component * @param scope scope * @return state map */ - StateMap getState(ProcessorNode processor, Scope scope); + StateMap getState(ConfigurableComponent component, Scope scope); /** - * Clears the state for the specified processor. + * Clears the state for the specified component. * - * @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 component component * @param scope scope - * @return state map */ - StateMap getState(ReportingTaskNode reportingTask, Scope scope); + void clearState(ConfigurableComponent component, 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 b79bd7648e2e..bef3f2db454d 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 @@ -18,8 +18,6 @@ import java.util.Set; -import org.apache.nifi.components.state.Scope; -import org.apache.nifi.components.state.StateMap; import org.apache.nifi.controller.ConfiguredComponent; import org.apache.nifi.controller.ScheduledState; import org.apache.nifi.controller.service.ControllerServiceNode; @@ -106,25 +104,10 @@ public interface ControllerServiceDAO { */ 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 b105c5584a2f..1896ad12ec86 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 @@ -16,8 +16,6 @@ */ 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.web.api.dto.ProcessorDTO; @@ -85,25 +83,10 @@ public interface ProcessorDAO { */ void deleteProcessor(String processorId); - /** - * Gets the specified processor. - * - * @param processorId processor id - * @return state map - */ - StateMap getState(String processorId, Scope scope); - /** * Verifies the processor can clear state. * * @param processorId processor id */ void verifyClearState(String processorId); - - /** - * Clears the state of the specified processor. - * - * @param processorId processor id - */ - void clearState(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 ab1a130c79fa..58c56801cd6d 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 @@ -18,8 +18,6 @@ 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; @@ -86,25 +84,10 @@ public interface ReportingTaskDAO { */ 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 index f0a9094daf54..a2ebd1a2fbd3 100644 --- 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 @@ -16,13 +16,12 @@ */ package org.apache.nifi.web.dao.impl; +import org.apache.nifi.components.ConfigurableComponent; +import org.apache.nifi.components.state.ExternalStateManager; 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; @@ -32,6 +31,16 @@ public class StandardComponentStateDAO implements ComponentStateDAO { private StateManagerProvider stateManagerProvider; + @Override + public StateMap getState(final ConfigurableComponent component, final Scope scope) { + switch (scope) { + case EXTERNAL: + return getExternalState(component); + default: + return getState(component.getIdentifier(), scope); + } + } + private StateMap getState(final String componentId, final Scope scope) { try { final StateManager manager = stateManagerProvider.getStateManager(componentId); @@ -45,49 +54,52 @@ private StateMap getState(final String componentId, final Scope scope) { } } - private void clearState(final String componentId) { + private StateMap getExternalState(final ConfigurableComponent component) { + if (component instanceof ExternalStateManager) { + try { + return ((ExternalStateManager)component).getExternalState(); + } catch (final IOException ioe) { + throw new IllegalStateException(String.format("Unable to get the external state for the specified component %s: %s", + component.getIdentifier(), ioe), ioe); + } + } + return null; + } + + @Override + public void clearState(final ConfigurableComponent component, final Scope scope) { + switch (scope) { + case EXTERNAL: + clearExternalState(component); + break; + default: + clearState(component.getIdentifier(), scope); + break; + } + } + + private void clearState(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)); } + manager.clear(scope); - // 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()); + private void clearExternalState(final ConfigurableComponent component) { + if (component instanceof ExternalStateManager) { + try { + ((ExternalStateManager)component).clearExternalState(); + } catch (final IOException ioe) { + throw new IllegalStateException(String.format("Unable to clear the external state for the specified component %s: %s", + component.getIdentifier(), ioe), ioe); + } + } } /* setters */ 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 099d4ece7184..44957c953092 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 @@ -16,8 +16,6 @@ */ package org.apache.nifi.web.dao.impl; -import org.apache.nifi.components.state.Scope; -import org.apache.nifi.components.state.StateMap; import org.apache.nifi.controller.ConfiguredComponent; import org.apache.nifi.controller.FlowController; import org.apache.nifi.controller.ScheduledState; @@ -30,7 +28,6 @@ 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; import java.util.ArrayList; @@ -44,7 +41,6 @@ public class StandardControllerServiceDAO extends ComponentDAO implements ControllerServiceDAO { private ControllerServiceProvider serviceProvider; - private ComponentStateDAO componentStateDAO; private FlowController flowController; private ControllerServiceNode locateControllerService(final String controllerServiceId) { @@ -293,33 +289,17 @@ public void deleteControllerService(final String controllerServiceId) { serviceProvider.removeControllerService(controllerService); } - @Override - public StateMap getState(final String controllerServiceId, final Scope scope) { - final ControllerServiceNode controllerService = locateControllerService(controllerServiceId); - return componentStateDAO.getState(controllerService, scope); - } - @Override public void verifyClearState(final String controllerServiceId) { final ControllerServiceNode controllerService = locateControllerService(controllerServiceId); controllerService.verifyCanClearState(); } - @Override - public void clearState(final String controllerServiceId) { - final ControllerServiceNode controllerService = locateControllerService(controllerServiceId); - componentStateDAO.clearState(controllerService); - } - /* setters */ public void setServiceProvider(final ControllerServiceProvider serviceProvider) { this.serviceProvider = serviceProvider; } - public void setComponentStateDAO(final ComponentStateDAO componentStateDAO) { - this.componentStateDAO = componentStateDAO; - } - public void setFlowController(final FlowController flowController) { this.flowController = flowController; } 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 ab9611717f4f..2d32dc84feb2 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 @@ -17,8 +17,6 @@ package org.apache.nifi.web.dao.impl; import org.apache.commons.lang3.StringUtils; -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; @@ -36,7 +34,6 @@ 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.quartz.CronExpression; import org.slf4j.Logger; @@ -56,7 +53,6 @@ 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(final String processorId) { final ProcessGroup rootGroup = flowController.getGroup(flowController.getRootGroupId()); @@ -444,29 +440,14 @@ public void deleteProcessor(String processorId) { } @Override - public StateMap getState(String processorId, final Scope scope) { - final ProcessorNode processor = locateProcessor(processorId); - return componentStateDAO.getState(processor, scope); - } - - @Override - public void verifyClearState(String processorId) { + public void verifyClearState(final String processorId) { final ProcessorNode processor = locateProcessor(processorId); processor.verifyCanClearState(); } - @Override - public void clearState(String processorId) { - final ProcessorNode processor = locateProcessor(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 e3cf89bd7569..df5a82f3044c 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,8 +24,6 @@ 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; @@ -38,14 +36,12 @@ 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 @@ -303,30 +299,15 @@ public void deleteReportingTask(String reportingTaskId) { 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/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 456b7eb81f28..48389c2c4d6e 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 @@ -86,16 +86,13 @@ - - - @@ -165,6 +162,7 @@ + diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/java/org/apache/nifi/web/ContentViewerController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/java/org/apache/nifi/web/ContentViewerController.java index 00dc06fa485a..f1f4b89f9a29 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/java/org/apache/nifi/web/ContentViewerController.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/java/org/apache/nifi/web/ContentViewerController.java @@ -108,7 +108,7 @@ protected void doGet(final HttpServletRequest request, final HttpServletResponse viewerContext.getRequestDispatcher("/message").forward(request, response); return; } catch (final AccessDeniedException ade) { - request.setAttribute("title", "Acess Denied"); + request.setAttribute("title", "Access Denied"); request.setAttribute("messages", "Unable to approve access to the specified content: " + ade.getMessage()); // forward to the error page 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 index c92942b50540..143d1d6da341 100644 --- 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 @@ -145,7 +145,7 @@ nf.ComponentState = (function () { * * @param {object} componentState */ - var loadComponentState = function (localState, clusterState) { + var loadComponentState = function (localState, clusterState, externalState) { var count = 0; var totalEntries = 0; var showPartialDetails = false; @@ -156,34 +156,35 @@ nf.ComponentState = (function () { // 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)); - }); - totalEntries += localState.totalEntryCount; + var addComponentStateData = function (componentState, getScope) { + if (nf.Common.isDefinedAndNotNull(componentState) + && nf.Common.isDefinedAndNotNull(componentState.state)) { + + $.each(componentState.state, function (i, stateEntry) { + componentStateData.addItem($.extend({ + id: count++, + scope: getScope(stateEntry), + }, stateEntry)); + }); + totalEntries += componentState.totalEntryCount; - if (nf.Common.isDefinedAndNotNull(localState.state) && localState.totalEntryCount !== localState.state.length) { - showPartialDetails = true; + if (componentState.totalEntryCount !== componentState.state.length) { + showPartialDetails = true; + } } } - if (nf.Common.isDefinedAndNotNull(clusterState)) { - $.each(clusterState.state, function (i, stateEntry) { - componentStateData.addItem($.extend({ - id: count++, - scope: 'Cluster' - }, stateEntry)); - }); - totalEntries += clusterState.totalEntryCount; + addComponentStateData(localState, function(stateEntry) { + return stateEntry.clusterNodeAddress; + }); - if (nf.Common.isDefinedAndNotNull(clusterState.state) && clusterState.totalEntryCount !== clusterState.state.length) { - showPartialDetails = true; - } - } + addComponentStateData(clusterState, function(stateEntry) { + return 'Cluster'; + }); + + addComponentStateData(externalState, function(stateEntry) { + return 'External' + (stateEntry.clusterNodeAddress ? ' - ' + stateEntry.clusterNodeAddress : ''); + }); // complete the update componentStateData.endUpdate(); @@ -273,11 +274,19 @@ nf.ComponentState = (function () { url: componentEntity.uri + '/state/clear-requests', dataType: 'json' }).done(function (response) { - // clear the table - clearTable(); + if (response.cleared) { + // clear the table + clearTable(); + + // reload the table with no state + loadComponentState() + } else { + nf.Dialog.showOkDialog({ + headerText: 'Failed to clear state', + dialogContent: response.message + }); + } - // reload the table with no state - loadComponentState() }).fail(nf.Common.handleAjaxError); } else { nf.Dialog.showOkDialog({ @@ -294,17 +303,6 @@ nf.ComponentState = (function () { {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, @@ -379,8 +377,25 @@ nf.ComponentState = (function () { var componentState = response.componentState; var componentStateTable = $('#component-state-table'); + // conditionally show the cluster node identifier + var gridInstance = componentStateTable.data().gridInstance; + var componentStateColumns = gridInstance.getColumns(); + if (nf.Canvas.isClustered() || nf.Common.isDefinedAndNotNull(componentState.externalState)) { + if (componentStateColumns.length === 2) { + // Show scope column + componentStateColumns.push({id: 'scope', field: 'scope', name: 'Scope', sortable: true, resizable: true}); + gridInstance.setColumns(componentStateColumns); + } + } else { + if (componentStateColumns.length === 3) { + // Hide scope columns + componentStateColumns.pop(); + gridInstance.setColumns(componentStateColumns); + } + } + // load the table - loadComponentState(componentState.localState, componentState.clusterState); + loadComponentState(componentState.localState, componentState.clusterState, componentState.externalState); // populate the name/description $('#component-state-name').text(componentEntity.component.name); diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java index 76603059a20e..7855e0233aa8 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java @@ -38,9 +38,11 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.commons.lang3.StringUtils; import org.apache.nifi.annotation.behavior.DynamicProperty; 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.SupportsBatching; import org.apache.nifi.annotation.behavior.WritesAttribute; import org.apache.nifi.annotation.behavior.WritesAttributes; @@ -50,6 +52,10 @@ import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.Validator; +import org.apache.nifi.components.state.ExternalStateManager; +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StandardStateMap; +import org.apache.nifi.components.state.StateMap; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.ProcessContext; @@ -82,7 +88,12 @@ + " In the event a dynamic property represents a property that was already set as part of the static properties, its value wil be" + " overriden with warning message describing the override." + " For the list of available Kafka properties please refer to: http://kafka.apache.org/documentation.html#configuration.") -public class GetKafka extends AbstractProcessor { +@Stateful(scopes = {Scope.EXTERNAL}, description = "While consuming messages from a Kafka topic, GetKafka periodically commits" + + " its offset information based on Zookeeper Commit Frequency." + + " Offsets are persisted in Zookeeper in per consumer group ids and topic partitions manner," + + " so that the state of a consumer group can be retained across events such as consumer reconnect." + + " Once offsets are cleared, GetKafka will resume consuming messages based on Auto Offset Rest configuration when it restarts.") +public class GetKafka extends AbstractProcessor implements ExternalStateManager { public static final String SMALLEST = "smallest"; public static final String LARGEST = "largest"; @@ -184,6 +195,9 @@ public class GetKafka extends AbstractProcessor { private volatile long deadlockTimeout; private volatile ExecutorService executor; + private volatile String zookeeperConnectionString; + private volatile String groupId; + private volatile String topic; @Override protected List getSupportedPropertyDescriptors() { @@ -218,11 +232,10 @@ public Set getRelationships() { } public void createConsumers(final ProcessContext context) { - final String topic = context.getProperty(TOPIC).getValue(); final Properties props = new Properties(); - props.setProperty("zookeeper.connect", context.getProperty(ZOOKEEPER_CONNECTION_STRING).getValue()); - props.setProperty("group.id", context.getProperty(GROUP_ID).getValue()); + props.setProperty("zookeeper.connect", zookeeperConnectionString); + props.setProperty("group.id", groupId); props.setProperty("client.id", context.getProperty(CLIENT_NAME).getValue()); props.setProperty("auto.commit.interval.ms", String.valueOf(context.getProperty(ZOOKEEPER_COMMIT_DELAY).asTimePeriod(TimeUnit.MILLISECONDS))); props.setProperty("auto.offset.reset", context.getProperty(AUTO_OFFSET_RESET).getValue()); @@ -256,8 +269,7 @@ public void createConsumers(final ProcessContext context) { props.setProperty("consumer.timeout.ms", "1"); } - int partitionCount = KafkaUtils.retrievePartitionCountForTopic( - context.getProperty(ZOOKEEPER_CONNECTION_STRING).getValue(), context.getProperty(TOPIC).getValue()); + int partitionCount = KafkaUtils.retrievePartitionCountForTopic(zookeeperConnectionString, topic); final ConsumerConfig consumerConfig = new ConsumerConfig(props); consumer = Consumer.createJavaConsumerConnector(consumerConfig); @@ -267,12 +279,12 @@ public void createConsumers(final ProcessContext context) { int concurrentTaskToUse = context.getMaxConcurrentTasks(); if (context.getMaxConcurrentTasks() < partitionCount){ this.getLogger().warn("The amount of concurrent tasks '" + context.getMaxConcurrentTasks() + "' configured for " - + "this processor is less than the amount of partitions '" + partitionCount + "' for topic '" + context.getProperty(TOPIC).getValue() + "'. " + + "this processor is less than the amount of partitions '" + partitionCount + "' for topic '" + topic + "'. " + "Consider making it equal to the amount of partition count for most efficient event consumption."); } else if (context.getMaxConcurrentTasks() > partitionCount){ concurrentTaskToUse = partitionCount; this.getLogger().warn("The amount of concurrent tasks '" + context.getMaxConcurrentTasks() + "' configured for " - + "this processor is greater than the amount of partitions '" + partitionCount + "' for topic '" + context.getProperty(TOPIC).getValue() + "'. " + + "this processor is greater than the amount of partitions '" + partitionCount + "' for topic '" + topic + "'. " + "Therefore those tasks would never see a message. To avoid that the '" + partitionCount + "'(partition count) will be used to consume events"); } @@ -481,4 +493,59 @@ private void releaseFlowFile(FlowFile flowFile, ProcessSession session, Map partitionOffsets = KafkaUtils.retrievePartitionOffsets(zookeeperConnectionString, topic, groupId); + + return new StandardStateMap(partitionOffsets, System.currentTimeMillis()); + } + + private boolean isReadyToAccessState() { + return !StringUtils.isEmpty(zookeeperConnectionString) + && !StringUtils.isEmpty(topic) + && !StringUtils.isEmpty(groupId); + } + + @Override + public void clearExternalState() throws IOException { + if (!isReadyToAccessState()) { + return; + } + // Block onTrigger starts creating new consumer until clear offset finishes. + synchronized (this.consumerStreamsReady) { + KafkaUtils.clearPartitionOffsets(zookeeperConnectionString, topic, groupId); + } + } + + /** + * GetKafka overrides this method in order to capture processor's property values required when it retrieves + * its state managed externally at Kafka. Since view/clear state operation can be executed before onTrigger() is called, + * we need to capture these values as it's modified. This method is also called when NiFi restarts and loads configs, + * so users can access external states right after restart of NiFi. + * @param descriptor of the modified property + * @param oldValue non-null property value (previous) + * @param newValue the new property value or if null indicates the property + */ + @Override + public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) { + if (ZOOKEEPER_CONNECTION_STRING.equals(descriptor)) { + zookeeperConnectionString = newValue; + } else if (TOPIC.equals(descriptor)) { + topic = newValue; + } else if (GROUP_ID.equals(descriptor)) { + groupId = newValue; + } + } + } \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaUtils.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaUtils.java index 8ddea61907fe..0cb0952ab344 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaUtils.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaUtils.java @@ -1,7 +1,20 @@ package org.apache.nifi.processors.kafka; +import java.io.IOException; import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import kafka.client.ClientUtils; +import kafka.common.OffsetAndMetadata; +import kafka.common.TopicAndPartition; +import kafka.javaapi.OffsetFetchRequest; +import kafka.javaapi.OffsetFetchResponse; +import kafka.javaapi.OffsetCommitRequest; +import kafka.javaapi.OffsetCommitResponse; +import kafka.network.BlockingChannel; +import kafka.utils.ZkUtils; import org.I0Itec.zkclient.ZkClient; import org.I0Itec.zkclient.exception.ZkMarshallingError; /* @@ -26,13 +39,34 @@ import kafka.api.TopicMetadata; import kafka.utils.ZKStringSerializer; import scala.collection.JavaConversions; +import scala.collection.Seq; +import scala.collection.mutable.Buffer; + +import static scala.collection.JavaConversions.asJavaList; +import static scala.collection.JavaConversions.asScalaBuffer; /** - * Utility class to support interruction with Kafka internals. + * Utility class to support interaction with Kafka internals. * */ class KafkaUtils { + private static final int ZK_CONNECTION_TIMEOUT_MS = 30_000; + private static final int SOCKET_TIMEOUT_MS = 30_000; + private static final int RETRY_BACK_OFF_MS = 30_000; + private static final String DEFAULT_CLIENT_ID = ""; + + private static class StingSerializer implements ZkSerializer { + @Override + public byte[] serialize(Object o) throws ZkMarshallingError { + return ZKStringSerializer.serialize(o); + } + + @Override + public Object deserialize(byte[] bytes) throws ZkMarshallingError { + return ZKStringSerializer.deserialize(bytes); + } + } /** * Will retrieve the amount of partitions for a given Kafka topic. @@ -41,18 +75,8 @@ static int retrievePartitionCountForTopic(String zookeeperConnectionString, Stri ZkClient zkClient = null; try { - zkClient = new ZkClient(zookeeperConnectionString); - zkClient.setZkSerializer(new ZkSerializer() { - @Override - public byte[] serialize(Object o) throws ZkMarshallingError { - return ZKStringSerializer.serialize(o); - } - - @Override - public Object deserialize(byte[] bytes) throws ZkMarshallingError { - return ZKStringSerializer.deserialize(bytes); - } - }); + zkClient = new ZkClient(zookeeperConnectionString, ZK_CONNECTION_TIMEOUT_MS); + zkClient.setZkSerializer(new StingSerializer()); scala.collection.Set topicMetadatas = AdminUtils .fetchTopicMetadataFromZk(JavaConversions.asScalaSet(Collections.singleton(topicName)), zkClient); if (topicMetadatas != null && topicMetadatas.size() > 0) { @@ -71,4 +95,77 @@ public Object deserialize(byte[] bytes) throws ZkMarshallingError { } } + static Map retrievePartitionOffsets(final String zookeeperConnectionString, final String topicName, + final String groupName) throws IOException { + return executeConsumerGroupCommand(zookeeperConnectionString, topicName, groupName, "retrievePartitionOffsets", (channel, topicPartitions) -> { + // Use version 0 to retrieve offsets from Zookeeper. version 1 will get from Kafka. + channel.send(new OffsetFetchRequest(groupName, topicPartitions, (short)0, 0, DEFAULT_CLIENT_ID).underlying()); + final OffsetFetchResponse offsetFetchResponse = OffsetFetchResponse.readFrom(channel.receive().buffer()); + final Map state = offsetFetchResponse.offsets().entrySet().stream() + .filter(entry -> entry.getValue().offset() > -1) + .collect(Collectors.toMap(entry -> "partition:" + entry.getKey().partition(), entry -> String.valueOf(entry.getValue().offset()))); + return state; + }); + } + + static void clearPartitionOffsets(final String zookeeperConnectionString, final String topicName, final String groupName) throws IOException { + executeConsumerGroupCommand(zookeeperConnectionString, topicName, groupName, "clearPartitionOffsets", (channel, topicPartitions) -> { + final Map offsets = topicPartitions.stream() + .collect(Collectors.toMap(tp -> tp, tp -> new OffsetAndMetadata(-1, OffsetAndMetadata.NoMetadata(), -1L))); + + // Use version 0 to clear offsets from Zookeeper. version 1 will get from Kafka. + final OffsetCommitRequest request = new OffsetCommitRequest(groupName, offsets, 0, DEFAULT_CLIENT_ID, (short) 0); + channel.send(request.underlying()); + final OffsetCommitResponse response = OffsetCommitResponse.readFrom(channel.receive().buffer()); + if (response.hasError()) { + throw new IOException("Failed to clear offsets due to " + response.errors()); + } + + return true; + }); + } + + static T executeConsumerGroupCommand(final String zookeeperConnectionString, final String topicName, final String groupId, + final String commandName, final ConsumerGroupCommand command) throws IOException { + ZkClient zkClient = null; + BlockingChannel channel = null; + + try { + zkClient = new ZkClient(zookeeperConnectionString, ZK_CONNECTION_TIMEOUT_MS); + zkClient.setZkSerializer(new StingSerializer()); + + final Buffer topics = asScalaBuffer(Collections.singletonList(topicName)); + final scala.collection.mutable.Map> topicMap = ZkUtils.getPartitionsForTopics(zkClient, topics); + + final List topicPartitions = asJavaList(topicMap.get(topicName).get()).stream() + .map(p -> new TopicAndPartition(topicName, (Integer) p)) + .collect(Collectors.toList()); + + // Use version 0 to retrieve offsets from Zookeeper. version 1 will get from Kafka. + channel = ClientUtils.channelToOffsetManager(groupId, zkClient, SOCKET_TIMEOUT_MS, RETRY_BACK_OFF_MS); + + return command.execute(channel, topicPartitions); + + } catch (IOException e) { + throw e; + } catch (Exception e) { + throw new IOException("Failed to " + commandName + " for topic: " + topicName + " consumer group:" + groupId, e); + } finally { + try { + if (channel != null) { + channel.disconnect(); + } + } finally { + if (zkClient != null) { + zkClient.close(); + } + } + } + } + + private interface ConsumerGroupCommand { + T execute(final BlockingChannel channel, final List topicPartitions) throws IOException; + } + + } diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestGetKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestGetKafka.java index dfcf0d9623cf..adc5d9e4f418 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestGetKafka.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestGetKafka.java @@ -16,6 +16,7 @@ */ package org.apache.nifi.processors.kafka; +import java.io.IOException; import java.lang.reflect.Field; import java.util.ArrayList; import java.util.Iterator; @@ -37,6 +38,9 @@ import kafka.consumer.ConsumerIterator; import kafka.message.MessageAndMetadata; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.fail; + public class TestGetKafka { @BeforeClass @@ -166,4 +170,44 @@ public byte[] answer(InvocationOnMock invocation) throws Throwable { } } + @Test + public void testGetState() throws Exception { + final GetKafka processor = new GetKafka(); + final TestRunner runner = TestRunners.newTestRunner(processor); + + assertNull("State should be null when required properties are not specified.", processor.getExternalState()); + + runner.setProperty(GetKafka.ZOOKEEPER_CONNECTION_STRING, "0.0.0.0:invalid-port"); + runner.setProperty(GetKafka.TOPIC, "testX"); + + assertNull("State should be null when required properties are not specified.", processor.getExternalState()); + + runner.setProperty(GetKafka.GROUP_ID, "consumer-group-id"); + + try { + processor.getExternalState(); + fail("The processor should try to access Zookeeper and should fail since it can not connect."); + } catch (IOException e) { + } + } + + @Test + public void testClearState() throws Exception { + final GetKafka processor = new GetKafka(); + final TestRunner runner = TestRunners.newTestRunner(processor); + + // Clear doesn't do anything until required properties are set. + processor.clearExternalState(); + + runner.setProperty(GetKafka.ZOOKEEPER_CONNECTION_STRING, "0.0.0.0:invalid-port"); + runner.setProperty(GetKafka.TOPIC, "testX"); + runner.setProperty(GetKafka.GROUP_ID, "consumer-group-id"); + + try { + processor.clearExternalState(); + fail("The processor should try to access Zookeeper and should fail since it can not connect Zookeeper."); + } catch (IOException e) { + } + } + } diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/pom.xml b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/pom.xml index 53e309a920bd..2b3e5246dac4 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/pom.xml +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/pom.xml @@ -34,6 +34,10 @@ org.apache.nifi nifi-utils + + org.apache.nifi + nifi-expression-language + org.apache.kafka kafka-clients diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java index 8bae304d3033..e2dc224f80fe 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java @@ -33,6 +33,7 @@ import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyDescriptor.Builder; +import org.apache.nifi.components.PropertyValue; import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; import org.apache.nifi.processor.AbstractSessionFactoryProcessor; @@ -316,19 +317,33 @@ String buildTransitURI(String securityProtocol, String brokers, String topic) { * Builds Kafka {@link Properties} */ Properties buildKafkaProperties(ProcessContext context) { - Properties properties = new Properties(); + Properties properties = getDefaultKafkaProperties(); for (PropertyDescriptor propertyDescriptor : context.getProperties().keySet()) { - String pName = propertyDescriptor.getName(); - String pValue = propertyDescriptor.isExpressionLanguageSupported() - ? context.getProperty(propertyDescriptor).evaluateAttributeExpressions().getValue() - : context.getProperty(propertyDescriptor).getValue(); - if (pValue != null) { - if (pName.endsWith(".ms")) { // kafka standard time notation - pValue = String.valueOf(FormatUtils.getTimeDuration(pValue.trim(), TimeUnit.MILLISECONDS)); - } - properties.setProperty(pName, pValue); - } + setKafkaProperty(properties, propertyDescriptor, context.getProperty(propertyDescriptor)); } return properties; } + + /** + * Returns new Properties instance with default values. + * Subclasses can override this method to define default Kafka property values. + */ + protected Properties getDefaultKafkaProperties() { + return new Properties(); + } + + protected void setKafkaProperty(final Properties properties, final PropertyDescriptor propertyDescriptor, PropertyValue propertyValue) { + String pName = propertyDescriptor.getName(); + String pValue = propertyDescriptor.isExpressionLanguageSupported() + ? propertyValue.evaluateAttributeExpressions().getValue() + : propertyValue.getValue(); + if (pValue != null) { + if (pName.endsWith(".ms")) { // kafka standard time notation + pValue = String.valueOf(FormatUtils.getTimeDuration(pValue.trim(), TimeUnit.MILLISECONDS)); + } + properties.setProperty(pName, pValue); + } else { + properties.remove(pName); + } + } } diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java index 2ed2db9aea87..356731824222 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java @@ -27,22 +27,40 @@ import java.util.Map; import java.util.Properties; import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.kafka.clients.consumer.CommitFailedException; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.utils.Utils; 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.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.attribute.expression.language.StandardPropertyValue; import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.state.ExternalStateManager; +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StandardStateMap; +import org.apache.nifi.components.state.StateMap; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; @@ -54,7 +72,12 @@ @InputRequirement(Requirement.INPUT_FORBIDDEN) @CapabilityDescription("Consumes messages from Apache Kafka") @Tags({ "Kafka", "Get", "Ingest", "Ingress", "Topic", "PubSub", "Consume" }) -public class ConsumeKafka extends AbstractKafkaProcessor> { +@Stateful(scopes = {Scope.EXTERNAL}, description = "After consuming messages, ConsumeKafka commits its offset information to Kafka" + + " so that the state of a consumer group can be retained across events such as consumer reconnect." + + " Offsets can be cleared when there is no consumer subscribing with the same consumer group id." + + " It may take more than 30 seconds for a consumer group to become able to be cleared after it is stopped from NiFi." + + " Once offsets are cleared, ConsumeKafka will resume consuming messages based on Offset Reset configuration.") +public class ConsumeKafka extends AbstractKafkaProcessor> implements ExternalStateManager { static final AllowableValue OFFSET_EARLIEST = new AllowableValue("earliest", "earliest", "Automatically reset the offset to the earliest offset"); @@ -88,6 +111,8 @@ public class ConsumeKafka extends AbstractKafkaProcessor DESCRIPTORS; static final Set RELATIONSHIPS; @@ -98,6 +123,8 @@ public class ConsumeKafka extends AbstractKafkaProcessor buildKafkaResource(ProcessContext context, Pr this.demarcatorBytes = context.getProperty(MESSAGE_DEMARCATOR).isSet() ? context.getProperty(MESSAGE_DEMARCATOR).evaluateAttributeExpressions().getValue().getBytes(StandardCharsets.UTF_8) : null; - this.topic = context.getProperty(TOPIC).evaluateAttributeExpressions().getValue(); - this.brokers = context.getProperty(BOOTSTRAP_SERVERS).evaluateAttributeExpressions().getValue(); - Properties kafkaProperties = this.buildKafkaProperties(context); - kafkaProperties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); - kafkaProperties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); + kafkaProperties = this.buildKafkaProperties(context); - KafkaConsumer consumer = new KafkaConsumer<>(kafkaProperties); + final Consumer consumer = buildKafkaResource(); consumer.subscribe(Collections.singletonList(this.topic)); + + return consumer; + } + + private Consumer buildKafkaResource() { + return buildKafkaResource(null); + } + + /** + * Builds an instance of {@link KafkaConsumer}, but does not subscribe to a topic yet. + * @param clientIdSuffix This method creates new KafkaConsumer instance. + * If there's another KafkaConsumer instance is already connected, + * {@link javax.management.InstanceAlreadyExistsException} is thrown. + * Since external state manager methods are called from different thread than onTrigger(), + * it's possible multiple KafkaConsumer instances to connect Kafka at the same time. + * To avoid InstanceAlreadyExistsException, if clientIdSuffix is specified, + * this method updates clientId by appending clientIdSuffix to the original clientId. + */ + Consumer buildKafkaResource(final String clientIdSuffix) { + if (kafkaProperties == null) { + return null; + } + + Properties props = kafkaProperties; + if (!StringUtils.isEmpty(clientIdSuffix)) { + // Update client.id while keep other properties as it is. + props = new Properties(); + props.putAll(kafkaProperties); + final String clientId = kafkaProperties.getProperty(ConsumerConfig.CLIENT_ID_CONFIG) + clientIdSuffix; + props.put(ConsumerConfig.CLIENT_ID_CONFIG, clientId); + } + + KafkaConsumer consumer = new KafkaConsumer<>(props); + return consumer; } + /** * Will release flow file. Releasing of the flow file in the context of this * operation implies the following: @@ -252,4 +310,193 @@ private void releaseFlowFile(FlowFile flowFile, ProcessContext context, ProcessS this.getLogger().info("Successfully received {} from Kafka with {} messages in {} millis", new Object[] { flowFile, msgCount, executionDuration }); session.transfer(flowFile, REL_SUCCESS); } + + @Override + public ExternalStateScope getExternalStateScope() { + return ExternalStateScope.CLUSTER; + } + + @Override + public StateMap getExternalState() throws IOException { + + if (!isReadyToAccessState()) { + return null; + } + + final String groupId = kafkaProperties.getProperty(ConsumerConfig.GROUP_ID_CONFIG); + return submitConsumerGroupCommand("Fetch offsets", consumer -> { + final Map partitionOffsets = consumer.partitionsFor(topic).stream() + .map(p -> new TopicPartition(topic, p.partition())) + .map(tp -> new ImmutablePair<>(tp, consumer.committed(tp))) + .filter(tpo -> tpo.right != null) + .collect(Collectors.toMap(tpo -> + "partition:" + tpo.left.partition(), + tpo -> String.valueOf(tpo.right.offset()))); + + logger.info("Retrieved offsets from Kafka, topic={}, groupId={}, partitionOffsets={}", + topic, groupId, partitionOffsets); + + return new StandardStateMap(partitionOffsets, System.currentTimeMillis()); + }, null); + } + + private boolean isReadyToAccessState() { + return !StringUtils.isEmpty(topic) + && !StringUtils.isEmpty(brokers) + && kafkaProperties != null + && !StringUtils.isEmpty(kafkaProperties.getProperty(ConsumerConfig.GROUP_ID_CONFIG)); + } + + /** + *

Clear offsets stored in Kafka, by committing -1 as offset of each partitions of specified topic.

+ * + *

Kafka allows commitSync if one of following conditions are met, + * see kafka.coordinator.GroupCoordinator.handleCommitOffsets for detail: + *

    + *
  1. The consumer is a member of the consumer group. In this case, + * even if there's other consumers connecting Kafka, offsets can be updated. + * It's dangerous to clear offsets if there're active consumers. + * When consumer.subscribe() and poll() are called, the consumer will be a member of the consumer group.
  2. + * + *
  3. There's no connected consumer within the group, + * and Kafka GroupCoordinator has marked the group as dead. + * It's safer but can take longer.
  4. + *
+ * + *

The consumer group state transition is an async operation at Kafka group coordinator. + * Although clearExternalState() can only be called when the processor is stopped, + * the consumer group may not be fully removed at Kafka, in that case, CommitFailedException will be thrown.

+ * + *

Following log msg can be found when GroupCoordinator has marked the group as dead + * in kafka.out on a Kafka broker server, it can take more than 30 seconds: + *

[GroupCoordinator]: Group [gid] generation 1 is dead + * and removed (kafka.coordinator.GroupCoordinator)

+ * + */ + @Override + public void clearExternalState() throws IOException { + + if (!isReadyToAccessState()) { + return; + } + + synchronized (this) { + final String groupId = kafkaProperties.getProperty(ConsumerConfig.GROUP_ID_CONFIG); + final Boolean result = submitConsumerGroupCommand("Clear offsets", consumer -> { + + final Map freshOffsets = consumer.partitionsFor(topic).stream() + .map(p -> new TopicPartition(topic, p.partition())) + .collect(Collectors.toMap(tp -> tp, tp -> new OffsetAndMetadata(-1))); + + consumer.commitSync(freshOffsets); + return true; + + }, e -> { + if (e instanceof CommitFailedException) { + throw new IOException("The stopped consumer may not have been removed completely." + + " It can take more than 30 seconds." + + " or there are other consumers connected to the same consumer group. Retrying later may succeed.", e); + } + }); + + if (result) { + logger.info("Offset is successfully cleared from Kafka. topic={}, groupId={}", topic, groupId); + } + } + } + + private interface ConsumerGroupCommand { + T execute(final Consumer consumer) throws Exception; + } + + private interface ConsumerGroupCommandExceptionHandler { + void handle(Exception e) throws IOException; + } + + /** + * Submit a consumer group related Kafka command. + * External state fetch operations can be executed at the same time as onTrigger() is running in a different thread. + * However, Kafka Consumer instance can not be shared among threads, so, different Consumer instance is created + * and used in this method, so that uses can access state while this processor consuming messages from Kafka. + * Kafka's {@link Utils#getContextOrKafkaClassLoader()}uses current thread's context classloader to load Kafka classes + * such as ByteArrayDeserializer, but since external state operations are called through NiFi Web API, + * those threads has Jetty class loader as its context class loader which does not know Kafka jar. + * As a workaround, this method uses another thread with context classloader unset to load Kafka classes properly. + * + * @param commandName command name is used for log message to show what command outputs what log messages + * @param command a command to submit and execute + * @param exHander optional exception handler + * @param command's return type + * @return command's return value + */ + private T submitConsumerGroupCommand(final String commandName, final ConsumerGroupCommand command, + final ConsumerGroupCommandExceptionHandler exHander) throws IOException { + // Use different KafkaConsumer instance because it can only be used + // from the thread which created the instance. + final ExecutorService executorService = Executors.newFixedThreadPool(1); + final Future future = executorService.submit(() -> { + // To use Kafka's classloader. + Thread.currentThread().setContextClassLoader(null); + final Consumer consumer = buildKafkaResource("-temp-command"); + if (consumer == null) { + return null; + } + + try { + return command.execute(consumer); + } catch (Exception e) { + if (exHander == null) { + throw e; + } + exHander.handle(e); + + } finally { + consumer.close(); + } + + return null; + }); + + try { + return future.get(CONSUMER_GRP_CMD_TIMEOUT_SEC, TimeUnit.SECONDS); + } catch (InterruptedException|ExecutionException|TimeoutException e) { + final String msg = commandName + " failed due to " + e; + logger.error(msg, e); + throw new IOException(msg, e); + } finally { + future.cancel(true); + executorService.shutdown(); + } + } + + @Override + protected Properties getDefaultKafkaProperties() { + Properties props = new Properties(); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); + return props; + } + + /** + * ConsumerKafka overrides this method in order to capture processor's property values required when it retrieves + * its state managed externally at Kafka. Since view/clear state operation can be executed before onTrigger() is called, + * we need to capture these values as it's modified. This method is also called when NiFi restarts and loads configs, + * so users can access external states right after restart of NiFi. + * @param descriptor of the modified property + * @param oldValue non-null property value (previous) + * @param newValue the new property value or if null indicates the property + */ + @Override + public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) { + if (kafkaProperties == null) { + kafkaProperties = getDefaultKafkaProperties(); + } + setKafkaProperty(kafkaProperties, descriptor, new StandardPropertyValue(newValue, null)); + + if (TOPIC.equals(descriptor)) { + topic = kafkaProperties.getProperty(TOPIC.getName()); + } else if (BOOTSTRAP_SERVERS.equals(descriptor)) { + brokers = kafkaProperties.getProperty(BOOTSTRAP_SERVERS.getName()); + } + } } \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java index 374a91b8bce4..2264140e730e 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java @@ -17,16 +17,32 @@ package org.apache.nifi.processors.kafka.pubsub; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; import java.nio.charset.StandardCharsets; import java.util.List; - +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.nifi.components.state.StateMap; import org.apache.nifi.util.MockFlowFile; import org.apache.nifi.util.TestRunner; import org.apache.nifi.util.TestRunners; import org.junit.Test; +import org.mockito.Mockito; // The test is valid and should be ran when working on this module. @Ignore is // to speed up the overall build @@ -167,4 +183,72 @@ public void validateGetAllMessagesWithProvidedDemarcator() throws Exception { consumeKafka.close(); } + + @Test + public void testGetState() throws Exception { + + final ConsumeKafka consumeKafka = Mockito.spy(new ConsumeKafka()); + final TestRunner runner = TestRunners.newTestRunner(consumeKafka); + + assertNull("State should be null when required properties are not specified.", consumeKafka.getExternalState()); + + final String topicEL = "${literal('topic'):toUpper()}"; + final String topic = "TOPIC"; + + runner.setProperty(ConsumeKafka.BOOTSTRAP_SERVERS, "0.0.0.0:1234"); + runner.setProperty(ConsumeKafka.TOPIC, topicEL); + runner.setProperty(ConsumeKafka.GROUP_ID, "GroupId"); + + final Consumer consumer = mock(Consumer.class); + doReturn(consumer).when(consumeKafka).buildKafkaResource(eq("-temp-command")); + + final List partitionInfo = IntStream.range(0, 3).mapToObj(p -> new PartitionInfo(topic, p, null, null, null)) + .collect(Collectors.toList()); + doReturn(partitionInfo).when(consumer).partitionsFor(eq(topic)); + + doReturn(new OffsetAndMetadata(100)) + .doReturn(new OffsetAndMetadata(101)) + .doReturn(new OffsetAndMetadata(102)) + .when(consumer).committed(any(TopicPartition.class)); + + final StateMap state = consumeKafka.getExternalState(); + + assertEquals(3, state.toMap().size()); + assertEquals(state.get("partition:0"), "100"); + assertEquals(state.get("partition:1"), "101"); + assertEquals(state.get("partition:2"), "102"); + } + + @Test + public void testClearState() throws Exception { + + final ConsumeKafka consumeKafka = Mockito.spy(new ConsumeKafka()); + final TestRunner runner = TestRunners.newTestRunner(consumeKafka); + + assertNull("State should be null when required properties are not specified.", consumeKafka.getExternalState()); + + final String topic = "topic"; + runner.setProperty(ConsumeKafka.BOOTSTRAP_SERVERS, "0.0.0.0:1234"); + runner.setProperty(ConsumeKafka.TOPIC, topic); + runner.setProperty(ConsumeKafka.GROUP_ID, "GroupId"); + + final Consumer consumer = mock(Consumer.class); + doReturn(consumer).when(consumeKafka).buildKafkaResource(eq("-temp-command")); + + final List partitionInfo = IntStream.range(0, 3).mapToObj(p -> new PartitionInfo(topic, p, null, null, null)) + .collect(Collectors.toList()); + doReturn(partitionInfo).when(consumer).partitionsFor(eq(topic)); + + doAnswer(invocation -> { + Map committedOffsets = (Map)invocation.getArguments()[0]; + assertEquals(3, committedOffsets.size()); + committedOffsets.values().stream().forEach(offset -> assertEquals("Offset should be cleared with -1", -1, offset.offset())); + return null; + }).when(consumer).commitSync(any()); + + consumeKafka.clearExternalState(); + + verify(consumer).commitSync(any()); + } + } From 6435fa0f9a4712662b619eda4f29329e84e600d7 Mon Sep 17 00:00:00 2001 From: Koji Kawamura Date: Sun, 31 Jul 2016 16:17:03 +0900 Subject: [PATCH 2/3] NIFI-2078, 2441: State management NIFI-2078: - Addressed feedback, handle default values properly in cace property value is not changed by user NIFI-2441: - Removed permission check from table record sort comparator --- .../webapp/js/nf/canvas/nf-component-state.js | 17 +++-------------- .../apache/nifi/processors/kafka/GetKafka.java | 6 +++++- .../nifi/processors/kafka/TestGetKafka.java | 4 ++-- .../kafka/pubsub/AbstractKafkaProcessor.java | 3 ++- .../processors/kafka/pubsub/ConsumeKafka.java | 4 ++-- 5 files changed, 14 insertions(+), 20 deletions(-) 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 index 143d1d6da341..df61abac1c7e 100644 --- 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 @@ -90,20 +90,9 @@ nf.ComponentState = (function () { var sort = function (sortDetails, data) { // defines a function for sorting var comparer = function (a, b) { - if(a.permissions.canRead && b.permissions.canRead) { - var aString = nf.Common.isDefinedAndNotNull(a.component[sortDetails.columnId]) ? a.component[sortDetails.columnId] : ''; - var bString = nf.Common.isDefinedAndNotNull(b.component[sortDetails.columnId]) ? b.component[sortDetails.columnId] : ''; - return aString === bString ? 0 : aString > bString ? 1 : -1; - } else { - if (!a.permissions.canRead && !b.permissions.canRead){ - return 0; - } - if(a.permissions.canRead){ - return 1; - } else { - return -1; - } - } + 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 diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java index 7855e0233aa8..1b88dc5df349 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java @@ -205,10 +205,14 @@ protected List getSupportedPropertyDescriptors() { .fromPropertyDescriptor(CLIENT_NAME) .defaultValue("NiFi-" + getIdentifier()) .build(); + final String defaultGroupId = getIdentifier(); final PropertyDescriptor groupIdWithDefault = new PropertyDescriptor.Builder() .fromPropertyDescriptor(GROUP_ID) - .defaultValue(getIdentifier()) + .defaultValue(defaultGroupId) .build(); + if (StringUtils.isBlank(groupId)) { + groupId = defaultGroupId; + } final List props = new ArrayList<>(); props.add(ZOOKEEPER_CONNECTION_STRING); diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestGetKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestGetKafka.java index adc5d9e4f418..fda896db13f1 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestGetKafka.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestGetKafka.java @@ -178,11 +178,11 @@ public void testGetState() throws Exception { assertNull("State should be null when required properties are not specified.", processor.getExternalState()); runner.setProperty(GetKafka.ZOOKEEPER_CONNECTION_STRING, "0.0.0.0:invalid-port"); - runner.setProperty(GetKafka.TOPIC, "testX"); + runner.setProperty(GetKafka.GROUP_ID, "consumer-group-id"); assertNull("State should be null when required properties are not specified.", processor.getExternalState()); - runner.setProperty(GetKafka.GROUP_ID, "consumer-group-id"); + runner.setProperty(GetKafka.TOPIC, "testX"); try { processor.getExternalState(); diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java index e2dc224f80fe..26256a6295cd 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/AbstractKafkaProcessor.java @@ -69,6 +69,7 @@ abstract class AbstractKafkaProcessor extends AbstractSessi static final AllowableValue SEC_SASL_PLAINTEXT = new AllowableValue("SASL_PLAINTEXT", "SASL_PLAINTEXT", "SASL_PLAINTEXT"); static final AllowableValue SEC_SASL_SSL = new AllowableValue("SASL_SSL", "SASL_SSL", "SASL_SSL"); + protected static final String DEFAULT_BOOTSTRAP_SERVERS = "localhost:9092"; static final PropertyDescriptor BOOTSTRAP_SERVERS = new PropertyDescriptor.Builder() .name(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG) .displayName("Kafka Brokers") @@ -77,7 +78,7 @@ abstract class AbstractKafkaProcessor extends AbstractSessi .addValidator(StandardValidators.NON_BLANK_VALIDATOR) .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile(BROKER_REGEX))) .expressionLanguageSupported(true) - .defaultValue("localhost:9092") + .defaultValue(DEFAULT_BOOTSTRAP_SERVERS) .build(); static final PropertyDescriptor CLIENT_ID = new PropertyDescriptor.Builder() .name(ProducerConfig.CLIENT_ID_CONFIG) diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java index 356731824222..f08e66e6b0f3 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java @@ -121,7 +121,7 @@ public class ConsumeKafka extends AbstractKafkaProcessor Date: Mon, 1 Aug 2016 19:59:31 +0900 Subject: [PATCH 3/3] NIFI-2078, NIFI-2442: Support full EL functionality. - Added ExternalStateManager.validateExternalStateAccess method to support full functionality of Expression Language, including Variable registory. - Changed how to capture property values from onPropertyModified to validateExternalStateAccess - Removed Expression Language dependency - Changed error handling to display validation errors appropriately --- .../state/ExternalStateManager.java | 15 ++++ .../src/main/asciidoc/developer-guide.adoc | 10 ++- .../apache/nifi/util/MockProcessContext.java | 5 ++ .../util/StandardProcessorTestRunner.java | 11 ++- .../java/org/apache/nifi/util/TestRunner.java | 7 ++ .../ComponentStateEndpointMerger.java | 22 +++++ .../nifi/web/StandardNiFiServiceFacade.java | 57 +++++++++--- .../nifi/web/controller/ControllerFacade.java | 11 +++ .../nifi/web/dao/ComponentStateDAO.java | 6 +- .../dao/impl/StandardComponentStateDAO.java | 37 +++----- .../nifi/processors/kafka/GetKafka.java | 62 +++++++------ .../nifi/processors/kafka/TestGetKafka.java | 63 ++++++------- .../nifi-kafka-pubsub-processors/pom.xml | 4 - .../processors/kafka/pubsub/ConsumeKafka.java | 88 +++++++++---------- .../kafka/pubsub/ConsumeKafkaTest.java | 55 +++++++++--- 15 files changed, 283 insertions(+), 170 deletions(-) diff --git a/nifi-api/src/main/java/org/apache/nifi/components/state/ExternalStateManager.java b/nifi-api/src/main/java/org/apache/nifi/components/state/ExternalStateManager.java index 896c3e349010..59392284e659 100644 --- a/nifi-api/src/main/java/org/apache/nifi/components/state/ExternalStateManager.java +++ b/nifi-api/src/main/java/org/apache/nifi/components/state/ExternalStateManager.java @@ -18,8 +18,11 @@ package org.apache.nifi.components.state; import org.apache.nifi.annotation.behavior.Stateful; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; import java.io.IOException; +import java.util.Collection; /** *

@@ -47,6 +50,18 @@ */ public interface ExternalStateManager { + /** + * To access an external system from those method implementations, configured property values of the component will be needed, + * such as Database connection URL or Kafka broker address. + * NiFi framework will call {@link #validateExternalStateAccess} method, to determine if required properties are set + * to access an external system, before {@link #getExternalState} or {@link #clearExternalState} is called. + * In this method, implementations has to validate configured properties, + * and also capture configured property values to use at getExternalState and clearExternalState. + * @param context validation context + * @return validation error result + */ + Collection validateExternalStateAccess(final ValidationContext context); + /** * Returns the current state for the component. This return value may be null. * diff --git a/nifi-docs/src/main/asciidoc/developer-guide.adoc b/nifi-docs/src/main/asciidoc/developer-guide.adoc index 84605fdfb54b..32c6f31afd3d 100644 --- a/nifi-docs/src/main/asciidoc/developer-guide.adoc +++ b/nifi-docs/src/main/asciidoc/developer-guide.adoc @@ -645,12 +645,14 @@ External State Manager is a mechanism to provide DFMs such interactions with ext If a Processor, Controller Service or Reporting Task interact with an external system that manages NiFi component state, it's advisable for those components to implement `ExternalStateManager` interface, and annotate the class with `@Stateful` using `Scope.EXTERNAL`. -By doing so, NiFi UI lets a DFM to access component states by calling corresponding `getExternalState` or `clearExternalState` method. +By doing so, NiFi UI lets a DFM access component states by calling corresponding `getExternalState` or `clearExternalState` method. -To access an external system from those method implementations, you will need configured property values of the component, +To access an external system from those method implementations, configured property values of the component will be needed, such as Database connection URL or Kafka broker address. -Since `ExternalStateManager` methods can be called before component is fully configured or scheduled, -it's important to use `onPropertyModified` to capture those required configuration values for an external system connection. +NiFi framework will call `validateExternalStateAccess` method, to determine if required properties are set to access an external system, +before `getExternalState` or `clearExternalState` is called. +At `valiadteExternalStateAccess` method, implementations has to validate configured properties, +and also capture configured property values to use at `getExternalState` and `clearExternalState`. `ExternalStateManager` also has the `getExternalStateScope` method which return either `ExternalStateScope.NODE` or `ExternalStateScope.CLUSTER` to indicate how a NiFi cluster should access the target external system. 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 63a5c8571ef3..6c15b1974ffa 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 @@ -32,6 +32,7 @@ import org.apache.nifi.components.ConfigurableComponent; 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.StateManager; import org.apache.nifi.controller.ControllerService; @@ -394,4 +395,8 @@ public void setPrimaryNode(boolean primaryNode) { } isPrimaryNode = primaryNode; } + + public ValidationContext newValidationContext() { + return new MockValidationContext(this, stateManager, variableRegistry); + } } 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 6607e85498e5..1845fe7db32d 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 @@ -606,7 +606,7 @@ public void assertNotValid(final ControllerService service) { throw new IllegalStateException("Controller Service has not been added to this TestRunner via the #addControllerService method"); } - final ValidationContext validationContext = new MockValidationContext(context, serviceStateManager, variableRegistry).getControllerServiceValidationContext(service); + final ValidationContext validationContext = newValidationContext().getControllerServiceValidationContext(service); final Collection results = context.getControllerService(service.getIdentifier()).validate(validationContext); for (final ValidationResult result : results) { @@ -625,7 +625,7 @@ public void assertValid(final ControllerService service) { throw new IllegalStateException("Controller Service has not been added to this TestRunner via the #addControllerService method"); } - final ValidationContext validationContext = new MockValidationContext(context, serviceStateManager, variableRegistry).getControllerServiceValidationContext(service); + final ValidationContext validationContext = newValidationContext().getControllerServiceValidationContext(service); final Collection results = context.getControllerService(service.getIdentifier()).validate(validationContext); for (final ValidationResult result : results) { @@ -740,7 +740,7 @@ public ValidationResult setProperty(final ControllerService service, final Prope final Map curProps = configuration.getProperties(); final Map updatedProps = new HashMap<>(curProps); - final ValidationContext validationContext = new MockValidationContext(context, serviceStateManager, variableRegistry).getControllerServiceValidationContext(service); + final ValidationContext validationContext = newValidationContext().getControllerServiceValidationContext(service); final ValidationResult validationResult = property.validate(value, validationContext); updatedProps.put(property, value); @@ -824,4 +824,9 @@ public void setClustered(boolean clustered) { public void setPrimaryNode(boolean primaryNode) { context.setPrimaryNode(primaryNode); } + + @Override + public ValidationContext newValidationContext() { + return context.newValidationContext(); + } } 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 023ef643594a..8d3ca63d9ed8 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 @@ -24,6 +24,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.controller.ControllerService; import org.apache.nifi.controller.queue.QueueSize; @@ -901,4 +902,10 @@ public interface TestRunner { * @param primaryNode Specify if this test emulates running as a primary node */ void setPrimaryNode(boolean primaryNode); + + /** + * Returns the {@link MockValidationContext} instance that enables custom validation test cases. + * @return newly created validation context + */ + ValidationContext newValidationContext(); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ComponentStateEndpointMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ComponentStateEndpointMerger.java index 84e6d706bf39..c8474d99ae9a 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ComponentStateEndpointMerger.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ComponentStateEndpointMerger.java @@ -17,6 +17,7 @@ package org.apache.nifi.cluster.coordination.http.endpoints; +import java.io.IOException; import java.net.URI; import java.util.ArrayList; import java.util.Arrays; @@ -30,12 +31,18 @@ import org.apache.nifi.cluster.protocol.NodeIdentifier; import org.apache.nifi.components.state.Scope; import org.apache.nifi.controller.state.SortedStateUtils; +import org.apache.nifi.stream.io.ByteArrayOutputStream; import org.apache.nifi.web.api.dto.ComponentStateDTO; import org.apache.nifi.web.api.dto.StateEntryDTO; import org.apache.nifi.web.api.dto.StateMapDTO; import org.apache.nifi.web.api.entity.ComponentStateEntity; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.ws.rs.core.StreamingOutput; public class ComponentStateEndpointMerger extends AbstractSingleDTOEndpoint { + private static final Logger logger = LoggerFactory.getLogger(ComponentStateEndpointMerger.class); public static final Pattern PROCESSOR_STATE_URI_PATTERN = Pattern.compile("/nifi-api/processors/[a-f0-9\\-]{36}/state"); public static final Pattern CONTROLLER_SERVICE_STATE_URI_PATTERN = Pattern.compile("/nifi-api/controller-services/[a-f0-9\\-]{36}/state"); public static final Pattern REPORTING_TASK_STATE_URI_PATTERN = Pattern.compile("/nifi-api/reporting-tasks/[a-f0-9\\-]{36}/state"); @@ -65,6 +72,21 @@ protected ComponentStateDTO getDto(ComponentStateEntity entity) { public void mergeResponses(ComponentStateDTO clientDto, Map dtoMap, Set successfulResponses, Set problematicResponses) { + // If there's a problematic response, pick it as the final response. + if (problematicResponses != null && problematicResponses.size() > 0) { + final NodeResponse problematicResponse = problematicResponses.iterator().next(); + if (problematicResponse.getResponse().getEntity() instanceof StreamingOutput) { + try ( + final ByteArrayOutputStream errResponse = new ByteArrayOutputStream(); + ) { + ((StreamingOutput)problematicResponse.getResponse().getEntity()).write(errResponse); + throw new IllegalStateException(errResponse.toString("utf-8")); + } catch (IOException e) { + logger.error("Failed to read problematic response due to {}", e, e); + } + } + } + // If there're more than 1 node returning external state, then it's a per node external state. final boolean externalPerNode = dtoMap.values().stream() .filter(component -> (component.getExternalState() != null && component.getExternalState().getState() != null)) 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 46bc9a16335a..d50949b01fef 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 @@ -203,6 +203,7 @@ import javax.ws.rs.WebApplicationException; import javax.ws.rs.core.Response; +import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; @@ -820,13 +821,19 @@ public CounterDTO updateCounter(final String counterId) { return dtoFactory.createCounterDto(controllerFacade.resetCounter(counterId)); } - private ClearComponentStateResultEntity clearComponentState(final ConfigurableComponent component) { + private ClearComponentStateResultEntity clearComponentState(final ConfiguredComponent configuredComponent, final ConfigurableComponent component) { logger.debug("clearComponentState: component={}, shouldManageClusterState={}, shouldManageExternalState={}", component, shouldManageClusterState(), shouldManageExternalState(component)); final ClearComponentStateResultEntity entity = new ClearComponentStateResultEntity(); - componentStateDAO.clearState(component, Scope.LOCAL); + try { + componentStateDAO.clearState(component, Scope.LOCAL); + } catch (IOException e) { + final String msg = String.format("Failed to clear state for %s due to %s", component, e); + logger.error(msg, e); + throw new IllegalStateException(msg, e); + } try { if (shouldManageClusterState()) { @@ -834,6 +841,10 @@ private ClearComponentStateResultEntity clearComponentState(final ConfigurableCo } if (shouldManageExternalState(component)) { + final Collection validationResults = controllerFacade.validateExternalStateAccess(configuredComponent, (ExternalStateManager) component); + if (!validationResults.isEmpty()) { + throw new IllegalStateException("Invalid configuration to access external state: " + validationResults); + } componentStateDAO.clearState(component, Scope.EXTERNAL); } } catch (Exception e) { @@ -843,7 +854,9 @@ private ClearComponentStateResultEntity clearComponentState(final ConfigurableCo // See: NodeClusterCoordinator.afterRequest() // Removing node doesn't help since it's related to external system, and other node will have the same issue. // So instead, return entity with message. - logger.warn("Failed to clear state for {} due to {}", component, e, e); + if (!(e instanceof IllegalStateException)) { + logger.error("Failed to clear state for {} due to {}", component, e, e); + } entity.setCleared(false); entity.setMessage(e.getMessage()); } @@ -857,7 +870,8 @@ public void verifyCanClearProcessorState(final String processorId) { @Override public ClearComponentStateResultEntity clearProcessorState(final String processorId) { - return clearComponentState(processorDAO.getProcessor(processorId).getProcessor()); + final ProcessorNode processor = processorDAO.getProcessor(processorId); + return clearComponentState(processor, processor.getProcessor()); } @Override @@ -867,7 +881,8 @@ public void verifyCanClearControllerServiceState(final String controllerServiceI @Override public ClearComponentStateResultEntity clearControllerServiceState(final String controllerServiceId) { - return clearComponentState(controllerServiceDAO.getControllerService(controllerServiceId).getControllerServiceImplementation()); + final ControllerServiceNode controllerService = controllerServiceDAO.getControllerService(controllerServiceId); + return clearComponentState(controllerService, controllerService.getControllerServiceImplementation()); } @Override @@ -877,7 +892,8 @@ public void verifyCanClearReportingTaskState(final String reportingTaskId) { @Override public ClearComponentStateResultEntity clearReportingTaskState(final String reportingTaskId) { - return clearComponentState(reportingTaskDAO.getReportingTask(reportingTaskId).getReportingTask()); + final ReportingTaskNode reportingTask = reportingTaskDAO.getReportingTask(reportingTaskId); + return clearComponentState(reportingTask, reportingTask.getReportingTask()); } @Override @@ -1948,7 +1964,7 @@ private boolean shouldManageExternalState(final ConfigurableComponent component) || controllerFacade.isPrimary()); } - private ComponentStateDTO getComponentState(final ConfigurableComponent component) { + private ComponentStateDTO getComponentState(final ConfiguredComponent configuredComponent, final ConfigurableComponent component) { logger.debug("getComponentState: component={}, shouldManageClusterState={}, shouldManageExternalState={}", component, shouldManageClusterState(), shouldManageExternalState(component)); @@ -1958,29 +1974,42 @@ private ComponentStateDTO getComponentState(final ConfigurableComponent componen logger.debug("clusterState={}", clusterState); final StateMap localState = componentStateDAO.getState(component, Scope.LOCAL); logger.debug("localState={}", localState); - final StateMap externalState = shouldManageExternalState(component) ? componentStateDAO.getState(component, Scope.EXTERNAL) : null; + StateMap externalState = null; + if (shouldManageExternalState(component)) { + final Collection validationResults = controllerFacade.validateExternalStateAccess(configuredComponent, (ExternalStateManager) component); + if (!validationResults.isEmpty()) { + throw new IllegalStateException("Invalid configuration to access external state: " + validationResults); + } else { + externalState = componentStateDAO.getState(component, Scope.EXTERNAL); + } + } logger.debug("externalState={}", externalState); return dtoFactory.createComponentStateDTO(component, localState, clusterState, externalState); - } catch (Exception e) { - logger.error("Failed to retrieve state for {} due to {}", component, e, e); - throw e; + } catch (IOException e) { + final String msg = String.format("Failed to clear state for %s due to %s", component, e); + logger.error(msg, e); + throw new IllegalStateException(msg, e); } + } @Override public ComponentStateDTO getProcessorState(final String processorId) { - return getComponentState(processorDAO.getProcessor(processorId).getProcessor()); + final ProcessorNode processor = processorDAO.getProcessor(processorId); + return getComponentState(processor, processor.getProcessor()); } @Override public ComponentStateDTO getControllerServiceState(final String controllerServiceId) { - return getComponentState(controllerServiceDAO.getControllerService(controllerServiceId).getControllerServiceImplementation()); + final ControllerServiceNode controllerService = controllerServiceDAO.getControllerService(controllerServiceId); + return getComponentState(controllerService, controllerService.getControllerServiceImplementation()); } @Override public ComponentStateDTO getReportingTaskState(final String reportingTaskId) { - return getComponentState(reportingTaskDAO.getReportingTask(reportingTaskId).getReportingTask()); + final ReportingTaskNode reportingTask = reportingTaskDAO.getReportingTask(reportingTaskId); + return getComponentState(reportingTask, reportingTask.getReportingTask()); } @Override 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 afc2659f47a8..c1bd2f78b2eb 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 @@ -33,10 +33,13 @@ import org.apache.nifi.cluster.coordination.ClusterCoordinator; import org.apache.nifi.cluster.protocol.NodeIdentifier; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.state.ExternalStateManager; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.Connection; import org.apache.nifi.connectable.Funnel; import org.apache.nifi.connectable.Port; +import org.apache.nifi.controller.ConfiguredComponent; import org.apache.nifi.controller.ContentAvailability; import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.Counter; @@ -69,6 +72,7 @@ import org.apache.nifi.processor.DataUnit; import org.apache.nifi.processor.Processor; import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.StandardValidationContext; import org.apache.nifi.provenance.ProvenanceEventRecord; import org.apache.nifi.provenance.ProvenanceRepository; import org.apache.nifi.provenance.SearchableFields; @@ -1757,6 +1761,12 @@ private void addIfAppropriate(final String searchStr, final String value, final } } + public Collection validateExternalStateAccess(final ConfiguredComponent configuredComponent, final ExternalStateManager stateManager) { + final StandardValidationContext validationContext = new StandardValidationContext(flowController, configuredComponent.getProperties(), + configuredComponent.getAnnotationData(), null, configuredComponent.getIdentifier(), variableRegistry); + return stateManager.validateExternalStateAccess(validationContext); + } + /* * setters */ @@ -1791,4 +1801,5 @@ public void setBulletinRepository(BulletinRepository bulletinRepository) { public void setVariableRegistry(VariableRegistry variableRegistry) { this.variableRegistry = variableRegistry; } + } 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 index b244e04c05e2..1940b5696e2c 100644 --- 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 @@ -20,6 +20,8 @@ import org.apache.nifi.components.state.Scope; import org.apache.nifi.components.state.StateMap; +import java.io.IOException; + public interface ComponentStateDAO { /** @@ -29,7 +31,7 @@ public interface ComponentStateDAO { * @param scope scope * @return state map */ - StateMap getState(ConfigurableComponent component, Scope scope); + StateMap getState(ConfigurableComponent component, Scope scope) throws IOException; /** * Clears the state for the specified component. @@ -37,6 +39,6 @@ public interface ComponentStateDAO { * @param component component * @param scope scope */ - void clearState(ConfigurableComponent component, Scope scope); + void clearState(ConfigurableComponent component, Scope scope) throws IOException; } 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 index a2ebd1a2fbd3..4ee0ff59c59d 100644 --- 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 @@ -32,7 +32,7 @@ public class StandardComponentStateDAO implements ComponentStateDAO { private StateManagerProvider stateManagerProvider; @Override - public StateMap getState(final ConfigurableComponent component, final Scope scope) { + public StateMap getState(final ConfigurableComponent component, final Scope scope) throws IOException { switch (scope) { case EXTERNAL: return getExternalState(component); @@ -54,20 +54,15 @@ private StateMap getState(final String componentId, final Scope scope) { } } - private StateMap getExternalState(final ConfigurableComponent component) { + private StateMap getExternalState(final ConfigurableComponent component) throws IOException { if (component instanceof ExternalStateManager) { - try { - return ((ExternalStateManager)component).getExternalState(); - } catch (final IOException ioe) { - throw new IllegalStateException(String.format("Unable to get the external state for the specified component %s: %s", - component.getIdentifier(), ioe), ioe); - } + return ((ExternalStateManager)component).getExternalState(); } return null; } @Override - public void clearState(final ConfigurableComponent component, final Scope scope) { + public void clearState(final ConfigurableComponent component, final Scope scope) throws IOException { switch (scope) { case EXTERNAL: clearExternalState(component); @@ -78,27 +73,17 @@ public void clearState(final ConfigurableComponent component, final Scope scope) } } - private void clearState(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)); - } - manager.clear(scope); - - } catch (final IOException ioe) { - throw new IllegalStateException(String.format("Unable to clear the state for the specified component %s: %s", componentId, ioe), ioe); + private void clearState(final String componentId, final Scope scope) throws IOException { + 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)); } + manager.clear(scope); } - private void clearExternalState(final ConfigurableComponent component) { + private void clearExternalState(final ConfigurableComponent component) throws IOException { if (component instanceof ExternalStateManager) { - try { - ((ExternalStateManager)component).clearExternalState(); - } catch (final IOException ioe) { - throw new IllegalStateException(String.format("Unable to clear the external state for the specified component %s: %s", - component.getIdentifier(), ioe), ioe); - } + ((ExternalStateManager)component).clearExternalState(); } } diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java index 1b88dc5df349..f3d969665230 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java @@ -20,6 +20,7 @@ import java.io.OutputStream; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -51,6 +52,8 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.annotation.lifecycle.OnStopped; 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.ExternalStateManager; import org.apache.nifi.components.state.Scope; @@ -237,6 +240,10 @@ public Set getRelationships() { public void createConsumers(final ProcessContext context) { + zookeeperConnectionString = context.getProperty(ZOOKEEPER_CONNECTION_STRING).getValue(); + groupId = context.getProperty(GROUP_ID).getValue(); + topic = context.getProperty(TOPIC).getValue(); + final Properties props = new Properties(); props.setProperty("zookeeper.connect", zookeeperConnectionString); props.setProperty("group.id", groupId); @@ -503,53 +510,44 @@ public ExternalStateScope getExternalStateScope() { return ExternalStateScope.CLUSTER; } + @Override + public Collection validateExternalStateAccess(ValidationContext context) { + final Collection validationResults = validate(context); + if (validationResults.isEmpty()) { + + // Capture settings. + context.getProperties().entrySet().forEach(kv -> { + final PropertyDescriptor descriptor = kv.getKey(); + // If value hasn't been modified by user from the default value, the value is null. + final String v = StringUtils.isBlank(kv.getValue()) ? descriptor.getDefaultValue() : kv.getValue(); + if (ZOOKEEPER_CONNECTION_STRING.equals(descriptor)) { + zookeeperConnectionString = v; + } else if (TOPIC.equals(descriptor)) { + topic = v; + } else if (GROUP_ID.equals(descriptor)) { + groupId = v; + } + }); + } + + return validationResults; + } + @Override public StateMap getExternalState() throws IOException { // We don't have to synchronize with onTrigger here, // since it merely retrieves state from Zk using different channel, it doesn't affect consuming. - if (!isReadyToAccessState()) { - return null; - } final Map partitionOffsets = KafkaUtils.retrievePartitionOffsets(zookeeperConnectionString, topic, groupId); return new StandardStateMap(partitionOffsets, System.currentTimeMillis()); } - private boolean isReadyToAccessState() { - return !StringUtils.isEmpty(zookeeperConnectionString) - && !StringUtils.isEmpty(topic) - && !StringUtils.isEmpty(groupId); - } - @Override public void clearExternalState() throws IOException { - if (!isReadyToAccessState()) { - return; - } // Block onTrigger starts creating new consumer until clear offset finishes. synchronized (this.consumerStreamsReady) { KafkaUtils.clearPartitionOffsets(zookeeperConnectionString, topic, groupId); } } - /** - * GetKafka overrides this method in order to capture processor's property values required when it retrieves - * its state managed externally at Kafka. Since view/clear state operation can be executed before onTrigger() is called, - * we need to capture these values as it's modified. This method is also called when NiFi restarts and loads configs, - * so users can access external states right after restart of NiFi. - * @param descriptor of the modified property - * @param oldValue non-null property value (previous) - * @param newValue the new property value or if null indicates the property - */ - @Override - public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) { - if (ZOOKEEPER_CONNECTION_STRING.equals(descriptor)) { - zookeeperConnectionString = newValue; - } else if (TOPIC.equals(descriptor)) { - topic = newValue; - } else if (GROUP_ID.equals(descriptor)) { - groupId = newValue; - } - } - } \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestGetKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestGetKafka.java index fda896db13f1..014176d75384 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestGetKafka.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestGetKafka.java @@ -16,14 +16,16 @@ */ package org.apache.nifi.processors.kafka; -import java.io.IOException; import java.lang.reflect.Field; import java.util.ArrayList; +import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; import org.apache.log4j.BasicConfigurator; +import org.apache.nifi.components.ValidationResult; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.util.MockFlowFile; import org.apache.nifi.util.TestRunner; @@ -38,8 +40,9 @@ import kafka.consumer.ConsumerIterator; import kafka.message.MessageAndMetadata; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.fail; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; public class TestGetKafka { @@ -171,43 +174,43 @@ public byte[] answer(InvocationOnMock invocation) throws Throwable { } @Test - public void testGetState() throws Exception { + public void testValidateExternalStateAccess() throws Exception { final GetKafka processor = new GetKafka(); final TestRunner runner = TestRunners.newTestRunner(processor); - assertNull("State should be null when required properties are not specified.", processor.getExternalState()); - - runner.setProperty(GetKafka.ZOOKEEPER_CONNECTION_STRING, "0.0.0.0:invalid-port"); - runner.setProperty(GetKafka.GROUP_ID, "consumer-group-id"); - - assertNull("State should be null when required properties are not specified.", processor.getExternalState()); + Collection validationResults = processor.validateExternalStateAccess(runner.newValidationContext()); + // Group Id has default value. These two properties are required. + assertEquals(2, validationResults.size()); + final List explanations = validationResults.stream().map(r -> r.getExplanation()).collect(Collectors.toList()); + assertTrue(explanations.contains("Topic Name is required")); + assertTrue(explanations.contains("ZooKeeper Connection String is required")); + // Set required properties, validation passes, values are set + runner.setProperty(GetKafka.ZOOKEEPER_CONNECTION_STRING, "0.0.0.0:9092"); runner.setProperty(GetKafka.TOPIC, "testX"); - try { - processor.getExternalState(); - fail("The processor should try to access Zookeeper and should fail since it can not connect."); - } catch (IOException e) { - } - } - - @Test - public void testClearState() throws Exception { - final GetKafka processor = new GetKafka(); - final TestRunner runner = TestRunners.newTestRunner(processor); + validationResults = processor.validateExternalStateAccess(runner.newValidationContext()); + assertEquals(0, validationResults.size()); - // Clear doesn't do anything until required properties are set. - processor.clearExternalState(); + assertEquals("testX", getPrivateFieldValue(processor, "topic")); + assertEquals("0.0.0.0:9092", getPrivateFieldValue(processor, "zookeeperConnectionString")); + assertNotNull("Default groupId should be used", getPrivateFieldValue(processor, "groupId")); - runner.setProperty(GetKafka.ZOOKEEPER_CONNECTION_STRING, "0.0.0.0:invalid-port"); - runner.setProperty(GetKafka.TOPIC, "testX"); + // Set groupId runner.setProperty(GetKafka.GROUP_ID, "consumer-group-id"); - try { - processor.clearExternalState(); - fail("The processor should try to access Zookeeper and should fail since it can not connect Zookeeper."); - } catch (IOException e) { - } + validationResults = processor.validateExternalStateAccess(runner.newValidationContext()); + assertEquals(0, validationResults.size()); + + assertEquals("testX", getPrivateFieldValue(processor, "topic")); + assertEquals("0.0.0.0:9092", getPrivateFieldValue(processor, "zookeeperConnectionString")); + assertNotNull("consumer-group-id", getPrivateFieldValue(processor, "groupId")); + } + + private Object getPrivateFieldValue(GetKafka processor, String fieldName) throws NoSuchFieldException, IllegalAccessException { + final Field field = processor.getClass().getDeclaredField(fieldName); + field.setAccessible(true); + return field.get(processor); } } diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/pom.xml b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/pom.xml index 2b3e5246dac4..53e309a920bd 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/pom.xml +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/pom.xml @@ -34,10 +34,6 @@ org.apache.nifi nifi-utils - - org.apache.nifi - nifi-expression-language - org.apache.kafka kafka-clients diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java index f08e66e6b0f3..ce0f06bd1006 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java @@ -20,6 +20,7 @@ import java.io.OutputStream; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; @@ -54,9 +55,11 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.lifecycle.OnStopped; -import org.apache.nifi.attribute.expression.language.StandardPropertyValue; import org.apache.nifi.components.AllowableValue; 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.ExternalStateManager; import org.apache.nifi.components.state.Scope; import org.apache.nifi.components.state.StandardStateMap; @@ -123,7 +126,7 @@ public class ConsumeKafka extends AbstractKafkaProcessor validateExternalStateAccess(ValidationContext context) { + final Collection validationResults = validate(context); + if (validationResults.isEmpty()) { + + // Capture settings. + context.getProperties().entrySet().forEach(kv -> { + final PropertyDescriptor descriptor = kv.getKey(); + // If value hasn't been modified by user from the default value, the value is null. + final String v = StringUtils.isBlank(kv.getValue()) ? descriptor.getDefaultValue() : kv.getValue(); + setKafkaProperty(kafkaProperties, descriptor, context.newPropertyValue(v)); + }); } + return validationResults; + } + + @Override + public StateMap getExternalState() throws IOException { + final String groupId = kafkaProperties.getProperty(ConsumerConfig.GROUP_ID_CONFIG); return submitConsumerGroupCommand("Fetch offsets", consumer -> { final Map partitionOffsets = consumer.partitionsFor(topic).stream() @@ -340,13 +366,6 @@ public StateMap getExternalState() throws IOException { }, null); } - private boolean isReadyToAccessState() { - return !StringUtils.isEmpty(topic) - && !StringUtils.isEmpty(brokers) - && kafkaProperties != null - && !StringUtils.isEmpty(kafkaProperties.getProperty(ConsumerConfig.GROUP_ID_CONFIG)); - } - /** *

Clear offsets stored in Kafka, by committing -1 as offset of each partitions of specified topic.

* @@ -376,10 +395,6 @@ private boolean isReadyToAccessState() { @Override public void clearExternalState() throws IOException { - if (!isReadyToAccessState()) { - return; - } - synchronized (this) { final String groupId = kafkaProperties.getProperty(ConsumerConfig.GROUP_ID_CONFIG); final Boolean result = submitConsumerGroupCommand("Clear offsets", consumer -> { @@ -393,7 +408,7 @@ public void clearExternalState() throws IOException { }, e -> { if (e instanceof CommitFailedException) { - throw new IOException("The stopped consumer may not have been removed completely." + + throw new IllegalStateException("The stopped consumer may not have been removed completely." + " It can take more than 30 seconds." + " or there are other consumers connected to the same consumer group. Retrying later may succeed.", e); } @@ -459,10 +474,16 @@ private T submitConsumerGroupCommand(final String commandName, final Consume try { return future.get(CONSUMER_GRP_CMD_TIMEOUT_SEC, TimeUnit.SECONDS); - } catch (InterruptedException|ExecutionException|TimeoutException e) { - final String msg = commandName + " failed due to " + e; - logger.error(msg, e); - throw new IOException(msg, e); + } catch (ExecutionException e) { + if (e.getCause() instanceof IOException) { + throw (IOException) e.getCause(); + } else if (e.getCause() instanceof RuntimeException) { + throw (RuntimeException) e.getCause(); + } else { + throw new IOException(commandName + " failed due to " + e, e); + } + } catch (InterruptedException|TimeoutException e) { + throw new IOException(commandName + " failed due to " + e, e); } finally { future.cancel(true); executorService.shutdown(); @@ -476,27 +497,4 @@ protected Properties getDefaultKafkaProperties() { props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); return props; } - - /** - * ConsumerKafka overrides this method in order to capture processor's property values required when it retrieves - * its state managed externally at Kafka. Since view/clear state operation can be executed before onTrigger() is called, - * we need to capture these values as it's modified. This method is also called when NiFi restarts and loads configs, - * so users can access external states right after restart of NiFi. - * @param descriptor of the modified property - * @param oldValue non-null property value (previous) - * @param newValue the new property value or if null indicates the property - */ - @Override - public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) { - if (kafkaProperties == null) { - kafkaProperties = getDefaultKafkaProperties(); - } - setKafkaProperty(kafkaProperties, descriptor, new StandardPropertyValue(newValue, null, null)); - - if (TOPIC.equals(descriptor)) { - topic = kafkaProperties.getProperty(TOPIC.getName()); - } else if (BOOTSTRAP_SERVERS.equals(descriptor)) { - brokers = kafkaProperties.getProperty(BOOTSTRAP_SERVERS.getName()); - } - } } \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java index 2264140e730e..b1223154e405 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-pubsub-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaTest.java @@ -27,7 +27,9 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; +import java.lang.reflect.Field; import java.nio.charset.StandardCharsets; +import java.util.Collection; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -37,6 +39,7 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; +import org.apache.nifi.components.ValidationResult; import org.apache.nifi.components.state.StateMap; import org.apache.nifi.util.MockFlowFile; import org.apache.nifi.util.TestRunner; @@ -187,24 +190,30 @@ public void validateGetAllMessagesWithProvidedDemarcator() throws Exception { @Test public void testGetState() throws Exception { - final ConsumeKafka consumeKafka = Mockito.spy(new ConsumeKafka()); + final ConsumeKafka processor = new ConsumeKafka(); + final ConsumeKafka consumeKafka = Mockito.spy(processor); final TestRunner runner = TestRunners.newTestRunner(consumeKafka); - assertNull("State should be null when required properties are not specified.", consumeKafka.getExternalState()); - final String topicEL = "${literal('topic'):toUpper()}"; final String topic = "TOPIC"; runner.setProperty(ConsumeKafka.BOOTSTRAP_SERVERS, "0.0.0.0:1234"); runner.setProperty(ConsumeKafka.TOPIC, topicEL); runner.setProperty(ConsumeKafka.GROUP_ID, "GroupId"); + runner.setProperty(processor.CLIENT_ID, "ClientId"); + + Collection validationResults = processor.validateExternalStateAccess(runner.newValidationContext()); + assertEquals(0, validationResults.size()); + + assertEquals(topic, getPrivateFieldValue(processor, "topic")); + assertEquals("0.0.0.0:1234", getPrivateFieldValue(processor, "brokers")); final Consumer consumer = mock(Consumer.class); doReturn(consumer).when(consumeKafka).buildKafkaResource(eq("-temp-command")); final List partitionInfo = IntStream.range(0, 3).mapToObj(p -> new PartitionInfo(topic, p, null, null, null)) .collect(Collectors.toList()); - doReturn(partitionInfo).when(consumer).partitionsFor(eq(topic)); + doReturn(partitionInfo).when(consumer).partitionsFor(any()); doReturn(new OffsetAndMetadata(100)) .doReturn(new OffsetAndMetadata(101)) @@ -222,22 +231,43 @@ public void testGetState() throws Exception { @Test public void testClearState() throws Exception { - final ConsumeKafka consumeKafka = Mockito.spy(new ConsumeKafka()); + System.setProperty("org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaTest.bootstrapServers", "set-from-env:9092"); + + final ConsumeKafka processor = new ConsumeKafka(); + final ConsumeKafka consumeKafka = Mockito.spy(processor); final TestRunner runner = TestRunners.newTestRunner(consumeKafka); - assertNull("State should be null when required properties are not specified.", consumeKafka.getExternalState()); + // Without required properties. + Collection validationResults = processor.validateExternalStateAccess(runner.newValidationContext()); + assertEquals(3, validationResults.size()); + final List explanations = validationResults.stream().map(r -> r.getExplanation()).collect(Collectors.toList()); + assertTrue(explanations.contains("topic is required")); + assertTrue(explanations.contains("client.id is required")); + assertTrue(explanations.contains("group.id is required")); + + // brokers has default value. + assertNull(getPrivateFieldValue(processor, "topic")); + assertEquals("localhost:9092", getPrivateFieldValue(processor, "brokers")); + // Set required properties final String topic = "topic"; - runner.setProperty(ConsumeKafka.BOOTSTRAP_SERVERS, "0.0.0.0:1234"); - runner.setProperty(ConsumeKafka.TOPIC, topic); - runner.setProperty(ConsumeKafka.GROUP_ID, "GroupId"); + runner.setProperty(processor.BOOTSTRAP_SERVERS, "${org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaTest.bootstrapServers}"); + runner.setProperty(processor.TOPIC, topic); + runner.setProperty(processor.GROUP_ID, "GroupId"); + runner.setProperty(processor.CLIENT_ID, "ClientId"); + + validationResults = processor.validateExternalStateAccess(runner.newValidationContext()); + assertEquals(0, validationResults.size()); + + assertEquals(topic, getPrivateFieldValue(processor, "topic")); + assertEquals("set-from-env:9092", getPrivateFieldValue(processor, "brokers")); final Consumer consumer = mock(Consumer.class); doReturn(consumer).when(consumeKafka).buildKafkaResource(eq("-temp-command")); final List partitionInfo = IntStream.range(0, 3).mapToObj(p -> new PartitionInfo(topic, p, null, null, null)) .collect(Collectors.toList()); - doReturn(partitionInfo).when(consumer).partitionsFor(eq(topic)); + doReturn(partitionInfo).when(consumer).partitionsFor(any()); doAnswer(invocation -> { Map committedOffsets = (Map)invocation.getArguments()[0]; @@ -251,4 +281,9 @@ public void testClearState() throws Exception { verify(consumer).commitSync(any()); } + private Object getPrivateFieldValue(ConsumeKafka processor, String fieldName) throws NoSuchFieldException, IllegalAccessException { + final Field field = processor.getClass().getDeclaredField(fieldName); + field.setAccessible(true); + return field.get(processor); + } }