NIFI-11556: Added ability to use a Process Group as a Stateless Flow

When shutting down FlowController, wait up to gracefulShutdownSeconds seconds for the components to stop before shutting down thread pools. This allows for asynchronous operations such as disableControllerServicesAsync to complete during shutdown. Updated StandardStatelessFlow so that on shutdown it catches more general Exception to ensure that shutdown succeeds

Ensure that Max Concurrent Tasks cannot be set less than 1 for stateless group; fixed typo in ProcessGroupDTO's docs; on shutdown, we may need to disable controller services asynchronously. At that point, the thread pool used to do so may already be shutdown. If so, catch this and create a new single-thread pool, disable the service, and immediately shutdown the pool. Also, if we fail to disable services on shutdown of a stateless flow, instead of throwing an Exception, just log it and move on - it doesn't make much sense for shutdown() to throw an Exception in that case.

Updated system tests so that when emptying a queue, we check the result and if the queue still has data (because a Processor hasn't acknowledged the data, for example) then continue issuing request until the queue fully becomes empty.

When shutting down input/output ports for stateless group, ensure that we wait for the ports' active threads to complete before returning

When stopping StatelessGroupNode, ensure that all the Processors, Controller Services (even those not executed by the Stateless Engine) are stopped/disabled before considering the Stateless Group to be fully STOPPED.

This closes #7253

Signed-off-by: David Handermann <exceptionfactory@apache.org>
This commit is contained in:
Mark Payne 2023-03-09 12:18:47 -05:00 committed by exceptionfactory
parent 84326ba4b2
commit b3c620411a
No known key found for this signature in database
GPG Key ID: 29B6A52D2AAE8DBA
181 changed files with 8871 additions and 966 deletions

View File

@ -0,0 +1,31 @@
/*
* 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;
public enum PortFunction {
/**
* Port functions as a standard port, transferring FlowFiles to all outgoing connections.
*/
STANDARD,
/**
* Port denotes that the invocation of the stateless flow has failed. If run using the Standard Engine, will operate as a Standard port. If running
* using the Stateless Engine, the transaction is rolled back, and any input FlowFiles will be transferred to this Port.
*/
FAILURE;
}

View File

@ -36,6 +36,7 @@ public class ProcessGroupStatus implements Cloneable {
private Integer outputCount;
private Long outputContentSize;
private Integer activeThreadCount;
private Integer statelessActiveThreadCount;
private Integer terminatedThreadCount;
private Integer queuedCount;
private Long queuedContentSize;
@ -152,6 +153,14 @@ public class ProcessGroupStatus implements Cloneable {
this.activeThreadCount = activeThreadCount;
}
public Integer getStatelessActiveThreadCount() {
return statelessActiveThreadCount;
}
public void setStatelessActiveThreadCount(final Integer statelessActiveThreadCount) {
this.statelessActiveThreadCount = statelessActiveThreadCount;
}
public Integer getTerminatedThreadCount() {
return terminatedThreadCount;
}
@ -427,6 +436,7 @@ public class ProcessGroupStatus implements Cloneable {
target.setBytesRead(target.getBytesRead() + toMerge.getBytesRead());
target.setBytesWritten(target.getBytesWritten() + toMerge.getBytesWritten());
target.setActiveThreadCount(target.getActiveThreadCount() + toMerge.getActiveThreadCount());
target.setStatelessActiveThreadCount(target.getStatelessActiveThreadCount() + toMerge.getStatelessActiveThreadCount());
target.setTerminatedThreadCount(target.getTerminatedThreadCount() + toMerge.getTerminatedThreadCount());
target.setFlowFilesTransferred(target.getFlowFilesTransferred() + toMerge.getFlowFilesTransferred());
target.setBytesTransferred(target.getBytesTransferred() + toMerge.getBytesTransferred());

View File

@ -0,0 +1,35 @@
/*
* 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.flow;
public enum ExecutionEngine {
/**
* Run using the standard NiFi engine
*/
STANDARD,
/**
* Run using the Stateless engine
*/
STATELESS,
/**
* Use the Execution Engine that is configured for the parent Process Group. If there is no parent Process Group, default to the standard engine.
*/
INHERITED;
}

View File

@ -18,12 +18,14 @@
package org.apache.nifi.flow;
import io.swagger.annotations.ApiModelProperty;
import org.apache.nifi.components.PortFunction;
public class VersionedPort extends VersionedComponent {
private PortType type;
private Integer concurrentlySchedulableTaskCount;
private ScheduledState scheduledState;
private Boolean allowRemoteAccess;
private PortFunction portFunction;
@ApiModelProperty("The number of tasks that should be concurrently scheduled for the port.")
public Integer getConcurrentlySchedulableTaskCount() {
@ -65,6 +67,15 @@ public class VersionedPort extends VersionedComponent {
this.allowRemoteAccess = allowRemoteAccess;
}
@ApiModelProperty("Specifies how the Port should function")
public PortFunction getPortFunction() {
return portFunction;
}
public void setPortFunction(final PortFunction portFunction) {
this.portFunction = portFunction;
}
@Override
public ComponentType getComponentType() {
if (type == PortType.OUTPUT_PORT) {

View File

@ -46,6 +46,10 @@ public class VersionedProcessGroup extends VersionedComponent {
private String defaultFlowFileExpiration;
private Long defaultBackPressureObjectThreshold;
private String defaultBackPressureDataSizeThreshold;
private ScheduledState scheduledState;
private ExecutionEngine executionEngine;
private Integer maxConcurrentTasks;
private String statelessFlowTimeout;
private String logFileSuffix;
@ -216,4 +220,40 @@ public class VersionedProcessGroup extends VersionedComponent {
public void setLogFileSuffix(final String logFileSuffix) {
this.logFileSuffix = logFileSuffix;
}
@ApiModelProperty("The Scheduled State of the Process Group, if the group is configured to use the Stateless Execution Engine. Otherwise, this value has no relevance.")
public ScheduledState getScheduledState() {
return scheduledState;
}
public void setScheduledState(final ScheduledState scheduledState) {
this.scheduledState = scheduledState;
}
@ApiModelProperty("The Execution Engine that should be used to run the components within the group.")
public ExecutionEngine getExecutionEngine() {
return executionEngine;
}
public void setExecutionEngine(final ExecutionEngine executionEngine) {
this.executionEngine = executionEngine;
}
@ApiModelProperty("The maximum number of concurrent tasks that should be scheduled for this Process Group when using the Stateless Engine")
public Integer getMaxConcurrentTasks() {
return maxConcurrentTasks;
}
public void setMaxConcurrentTasks(final Integer maxConcurrentTasks) {
this.maxConcurrentTasks = maxConcurrentTasks;
}
@ApiModelProperty("The maximum amount of time that the flow is allows to run using the Stateless engine before it times out and is considered a failure")
public String getStatelessFlowTimeout() {
return statelessFlowTimeout;
}
public void setStatelessFlowTimeout(final String timeout) {
this.statelessFlowTimeout = timeout;
}
}

View File

@ -46,6 +46,11 @@ public enum ComponentType {
*/
REPORTING_TASK,
/**
* Bulletin is associated with a Process Group
*/
PROCESS_GROUP,
/**
* Bulletin is associated with a Parameter Provider
*/

View File

@ -3405,6 +3405,7 @@ The contents of the _nifi.properties_ file are relatively stable but can change
Consider configuring items below marked with an asterisk (`*`) in such a way that upgrading will be easier. For example, change the default directory configurations to locations outside the main root installation. In this way, these items can remain in their configured location through an upgrade, allowing NiFi to find all the repositories and configuration files and pick up where it left off as soon as the old version is stopped and the new version is started. Furthermore, the administrator may reuse this _nifi.properties_ file and any other configuration files without having to re-configure them each time an upgrade takes place. See <<upgrading_nifi>> for more details.
[[core_properties]]
=== Core Properties +
The first section of the _nifi.properties_ file is for the Core Properties. These properties apply to the core framework as a whole.

Binary file not shown.

After

Width:  |  Height:  |  Size: 74 KiB

Binary file not shown.

Before

Width:  |  Height:  |  Size: 116 KiB

After

Width:  |  Height:  |  Size: 273 KiB

View File

@ -18,6 +18,8 @@
Apache NiFi Team <dev@nifi.apache.org>
:homepage: http://nifi.apache.org
:linkattrs:
:imagesdir: images
== Introduction
Apache NiFi is a dataflow system based on the concepts of flow-based programming. It supports
@ -127,7 +129,7 @@ UI may become unavailable.
You can use these archived files to rollback flow configuration. To do so, stop NiFi, replace _flow.xml.gz_ with a desired backup copy, then restart NiFi.
In a clustered environment, stop the entire NiFi cluster, replace the _flow.xml.gz_ of one of nodes, and restart the node. Remove _flow.xml.gz_ from other nodes.
Once you confirmed the node starts up as a one-node cluster, start the other nodes. The replaced flow configuration will be synchronized across the cluster.
The name and location of _flow.xml.gz_, and auto archive behavior are configurable. See the link:administration-guide.html#core-properties-br[System Administrators Guide] for further details.
The name and location of _flow.xml.gz_, and auto archive behavior are configurable. See the link:administration-guide.html#core_properties[System Administrators Guide] for further details.
@ -350,7 +352,10 @@ link:administration-guide.html[System Administrators Guide].
[[process_group]]
image:iconProcessGroup.png["Process Group", width=32]
*Process Group*: Process Groups can be used to logically group a set of components so that the dataflow is easier to understand
and maintain. When a Process Group is dragged onto the canvas, the DFM is prompted to name the Process Group. The Process Group will
and maintain. Additionally, Process Groups are used as a mechanism for grouping together components in such a way that they operate together
as a more cohesive unit. For example, by configuring the Execution Engine or the FlowFile Outbound Policy.
When a Process Group is dragged onto the canvas, the DFM is prompted to name the Process Group. The Process Group will
then be nested within that parent group.
Once you have dragged a Process Group onto the canvas, you can interact with it by right-clicking on the Process Group and selecting an option from the
@ -716,7 +721,18 @@ image::relationships-tab.png["Relationships Tab"]
In order for a Processor to be considered valid and able to run, each Relationship defined by the Processor must be either connected to a downstream component or auto-terminated. If a Relationship is auto-terminated, any FlowFile that is routed to that Relationship will be removed from the flow and its processing considered complete. Any Relationship that is already connected to a downstream component cannot be auto-terminated. The Relationship must first be removed from any Connection that uses it. Additionally, for any Relationship that is selected to be auto-terminated, the auto-termination status will be cleared (turned off) if the Relationship is added to a Connection.
===== Automatically Retry
Users can also configure whether or not FlowFiles routed to a given Relationship should be retried.
Users can also configure whether or not FlowFiles routed to a given Relationship should be retried. If a FlowFile is routed to any Relationship that is configured to be retried,
the FlowFile will be re-queued and the Processor will attempt to process it again. If the Processor routes the FlowFile to a retriable Relationship again (either the same Relationship
or another that is configured to be retried), it will be re-queued again, up to the number of
times specified by the user. If the Processor routes the FlowFile to a retriable Relationship after the specified number of retries, the FlowFile will be transferred to
the Connection(s) that include that Relationship - or auto-terminated, as configured.
If the Processor routes the FlowFile to any Relationship that is not configured to be retried, it will be routed to that Relationship immediately.
For example, consider a Processor with two relationships: `success` and `failure`.
A user configures the `failure` Relationship to retry 10 times and also be configured to auto-terminate. In this
case, if an incoming FlowFile is routed to the `failure` Relationship,
it will be retried up to 10 times. After 10 attempts, if it is routed to `failure` again, it will be auto-terminated. However, if at any point it is
routed to `success`, it will immediatley be transferred to the Connection(s) that include the `success` Relationship and not retried any further.
====== Number of Retry Attempts
For relationships set to retry, this number indicates how many times a FlowFile will attempt to reprocess before it is routed elsewhere.
@ -724,8 +740,11 @@ For relationships set to retry, this number indicates how many times a FlowFile
====== Retry Back Off Policy
When a FlowFile is to be retried, the user can configure the backoff policy with two options:
* Penalize - Retry attempts will occur in time, but the processor will continue to process other FlowFiles.
* Yield - No other FlowFile processing will occur until all retry attempts have been made.
* Penalize - Retry attempts will occur in time, but the processor will continue to process other FlowFiles in the meantime. The Processor will continue to run,
processing the FlowFiles are available to it.
* Yield - The Processor will not be scheduled to run again for some amount of time (determined by the "Yield Duration" configured in the Settings tab). This should be used when
either it is desirable to maintain the order of data in the queue and not process subsequent FlowFiles until the first has been processed, or when routing data to the configured
Relationship indicates that further processing of other data will not be successful.
====== Retry Maximum Back Off Period
Initial retries are based on the Penalty/Yield Duration time specified in the Settings tab. The duration time is repeatedly doubled for every subsequent retry attempt. This number indicates the maximum allowable time period before another retry attempt occurs.
@ -758,7 +777,10 @@ This tab contains several different configuration items. First is the Process Gr
The next configuration element is the Process Group Parameter Context, which is used to provide parameters to components of the flow. From this drop-down, the user is able to choose which Parameter Context should be bound to this Process Group and can optionally create a new one to bind to the Process Group. For more information refer to <<Parameters>> and <<parameter-contexts,Parameter Contexts>>.
The third element in the configuration dialog is the Process Group Comments. This provides a mechanism to add any useful information about the Process Group.
The next section provides configuration elements for determining how the Process Group should be scheduled to run. NiFi
supports two different Execution Engines: The Traditional Execution Engine, and the Stateless Execution Engine.
Additionally, the Execution Engine can be inherited from the parent Process Group, which is the default behavior.
See <<Execution_Engines, Execution Engines>> for more information.
The next two elements, Process Group FlowFile Concurrency and Process Group Outbound Policy, are covered in the following sections.
@ -862,6 +884,9 @@ root Process Group obtains its default back pressure settings from `nifi.propert
NOTE: Setting the Default FlowFile Expiration to a non-zero value may lead to data loss due to a FlowFile expiring as its time limit is reached.
The last element in the configuration dialog is the Process Group Comments. This provides a mechanism to add any useful information about the Process Group.
==== Controller Services
The Controller Services tab in the Process Group configuration dialog is covered in <<Controller_Services_for_Dataflows>>.
@ -1257,6 +1282,7 @@ NOTE: NiFi must be restarted for these updates to be picked up.
For more information, see the <<administration-guide.adoc#custom_properties,Custom Properties>> section in the System Administrator's Guide.
[[Controller_Services]]
=== Controller Services
@ -1581,7 +1607,7 @@ Using Site-to-Site provides the following benefits:
** Site-to-Site optionally makes use of Certificates in order to encrypt data and provide authentication and authorization. Each port can be configured
to allow only specific users, and only those users will be able to see that the port even exists. For information on configuring the Certificates,
see the
link:administration-guide.html#security-configuration[Security Configuration] section of the
link:administration-guide.html#security_configuration[Security Configuration] section of the
link:administration-guide.html[System Administrators Guide].
* Scalable
@ -1637,7 +1663,7 @@ link:administration-guide.html[System Administrators Guide].
[[Site-to-Site_Transport_Protocol]]
*Transport Protocol*: On a Remote Process Group creation or configuration dialog, you can choose Transport Protocol to use for Site-to-Site communication as shown in the following image:
image:configure-remote-process-group.png["Configure Remote Process Group"]
image::configure-remote-process-group.png["Configure Remote Process Group"]
By default, it is set to _RAW_ which uses raw socket communication using a dedicated port. _HTTP_ transport protocol is especially useful if the remote
NiFi instance is in a restricted network that only allow access through HTTP(S) protocol or only accessible from a specific HTTP Proxy server.
@ -1701,6 +1727,269 @@ image::simple-flow.png["Simple Flow"]
Now see the following section on how to start and stop the dataflow. When the dataflow is running, be sure to note the statistical information that is displayed on the face of each processor (see <<processor_anatomy>>).
[[Execution_Engines]]
== Execution Engines
NiFi supports two different Execution Engines: Traditional and Stateless.
The Execution Engine dictates how the components within a given Process Group are executed. The two Execution Engines differ in many ways.
Each has its own pros and cons, and as such NiFi allows the flow developer to configure which Execution Engine to use for individual Process Groups.
This section will highlight the differences between the two models.
=== Traditional Execution Engine
The Traditional Execution Engine is the default.
With the Traditional Engine, each component within the Process Group is scheduled independently. Queues are used to buffer data between Processors, and each Processor
operates on the data within its queue independently of other Processors.
The data that resides within a given queue is persisted to NiFi's repositories such that if NiFi is restarted, data will be restored to the
queue in which it resided before the restart.
=== Stateless Execution Engine
When a Process Group is configured to use the Stateless Execution Engine, the entire Process Group functions much more similarly to a Processor.
The Process Group is given a specified number of Concurrent Tasks. Each Concurrent Task is responsible for executing the entirety of the flow.
The source Processors (i.e., the Processors within the Process Group that have no incoming Connections) will be triggered to run once. If there are any Input
Ports for the Process Group, they will also have the opportunity to run and bring in a single FlowFile. At that point, the flow will run all the way to its
completion, if possible*, before any additional data is brought into the flow.
_*_ In some scenarios, the dataflow cannot complete without bringing in additional data. For example, a MergeContent Processor may require additional input
before it is able to proceed. In this case, the source components will be allowed to trigger again, until the component is capable of proceeding.
Data that resides in a queue within a Stateless group is not persisted. As a result, if NiFi is restarted, data will not be restored to the queue that it resided
in before the restart. Instead, the data that is being processed by the Stateless group is lost. This provides some interesting opportunities, which will be discussed below, in the
<<when_to_use_stateless_standard, When to Use Stateless vs Standard Execution Engine>>.
=== Comparing and Contrasting
Now that we have an overall understanding of the roles of the different Execution Engines, we will compare and contrast how each of the Execution Engines differ
in each of several important capabilities.
==== Scheduling
The Traditional Engine schedules each Processor individually. One Processor may be configured to run once every five minutes, while another Processor is scheduled to run
as fast as possible, using eight concurrent tasks. The primary job of the queue between Processors is to allow for the Processors to operate at different rates. If the source
runs faster than the destination, it will queue up data until the configured backpressure threshold is reached. Then, the source will stop producing data so that the destination
has the opportunity to catch up. If too many FlowFiles are queued up, the FlowFiles (i.e., the attributes, etc.) are swapped to disk in order to avoid exhausting the Java heap.
This approach gives the flow developer very fine grained control over how many resources to dedicate to each step of the processing throughout the pipeline.
In contrast, the Stateless Engine uses a single thread to run the entire flow from start to finish. The source components are triggered first in order to bring data into the flow.
If a FlowFile is brought into the flow, the destination of the queue where the data landed is triggered until it has worked off all of its data. Most of the time, this is a single
invocation of the Processor. The next Processor in the flow is then triggered to run, and so forth. Any backpressure that is configured for the queues is ignored. FlowFiles are never
swapped to disk. However, this is typically not a concern, as only a single FlowFile at a time is typically dealt with in a flow.
When configuring the Execution Engine to use for a Process Group, though, the flow developer is able to specify the Max number of Concurrent Tasks to use for the Stateless Engine.
If the value is set to N, greater than 1, there will be N copies of the Stateless flow run concurrently. For each copy of the flow, the source components will be triggered, and then
the next Processor in line, and so forth. It is important to note that if N is greater than 1, there will then be N copies of each Processor in the flow used to process data. Therefore,
if the Processor needs to store objects internally, such as MergeContent, each copy of the flow has a separate copy of that internal state. As a result, if MergeContent is configured to
require a minimum of 1,000 FlowFiles in order to merge and the flow is run with 8 Concurrent Tasks, each of the 8 flows may have up to 999 FlowFiles for a total of 7,992 FlowFiles without
merging any data. Therefore, flow designers should be cautious when using the Stateless Engine in conjunction with Processors that must wait on some condition before proceeding.
Stateless Flows may also contain source Processors that are not to be run continually. For example, a GetFile Processor may be the first Processor in a flow. The desire may be to run the Processor
only once every 5 minutes. This may complicate things, since Processors are not individually scheduled. To that end, all source Processors are checked in order to determine how often they should
be scheduled. Whichever component has the smallest Run Schedule (i.e., is scheduled the fastest) will determine how frequently the Stateless flow should be scheduled. For example, if there is a
ListFile Processor configured to run once every hour and a ListS3 Processor configured to run once every minute, both source Processors will be triggered to run once every minute.
When using the Stateless Engine, the Timer-Driven Scheduler is always used. CRON scheduling is not supported for Stateless flows at this time. If CRON scheduling is desired, it is often sufficient
to use the Traditional Engine for the CRON driven Processor and then connect it to a Stateless Process Group.
The Traditional Engine allows configuring a Run Duration for each individual Processor. This often greatly improves performance. This is not available when using a Stateless flow. With a Stateless
flow, this is instead managed for you automatically, depending on the Processors used in the flow. If any Processor in the flow requires being triggered serially, the Stateless Flow will not use
a Run Duration greater than a single invocation. Otherwise, it will run for up to 100 milliseconds. If any invocation produces a failure, the Run Duration is cut short.
==== Transactionality
Transactionality is perhaps the most stark difference between the Traditional Engine and the Stateless Engine. With the Traditional Engine, the Processor is the transaction boundary.
That is, a Processor performs some action on a FlowFile, and then the transaction is completed. The FlowFile is transferred to the next Processor in the flow, as the first Processor
performs any sort of cleanup.
For example, consider a processor consuming from a JMS Queue. With the Traditional Engine, the Processor would consume the message from JMS. Then, it would create a FlowFile based on the
JMS message, and transfer the FlowFile to the `success` Relationship and commit the session. At this point, the framework would ensure that the data has been persisted to the underlying
repositories and add the FlowFile to the outbound queue. Finally, the JMS consumer would be given the opportunity to acknowledge receipt of the message.
At this point, any number of things may happen to the FlowFile. It may be dropped. It may be transformed. It may be pushed to two different destinations. Or it may stay in the flow because some
downstream system, such as a database, is down. Regardless, though, the message has already been acknowledged and it will not be redelivered. Because of this, it is important that NiFi persist
that data so that upon restart it can finish processing the data.
One powerful advantage of this paradigm is that the source need not be persistent or transactional. For example, a client could make an HTTP POST request to NiFi to provide data. As long as
NiFi responds with a "200 OK" or a "201 Created" or similar message, it is okay to assume that the message will be processed according to the flow.
On the other hand, if the source of the data is, in fact, transactional, such as a JMS queue, this may be less desirable. If NiFi were to be shutdown, perhaps due to a hardware failure,
that data that was already acknowledged will not be redelivered. The NiFi instance owns the data and must be brought back up in order to finish processing the data.
However, the semantics change when the Stateless Engine is used instead. The transaction boundary is no longer the Processor but instead becomes the entire Process Group.
When the same flow is run using the Stateless Engine, the JMS consumer still receives a JMS message and creates a FlowFile from it. It still transfers the FlowFile to the `success`
Relationship and commits the session. With the Stateless Engine, though, instead of persisting data to the underlying repository and giving the JMS consumer the opportunity to acknowledge
receipt of the message, it instead just passes the FlowFile along to the next Processor. That Processor then handles the FlowFile and passes it on. The JMS consumer is not given the
opportunity to acknowledge the message until the FlowFile has run through the entire flow successfully.
Now, this provides a very different way of thinking of thinking about how to build the flow. It also offers some important new capabilities. For example, we how have the ability to have
a follow-on Processor indicate that there was a failure processing the data. In order to do this, an Output Port can be configured as a "Failure Port."
image::failure-port-config.png["Failure Port"]
Now, if any FlowFile is routed to this Port, the entire transaction is rolled back. Any FlowFiles in the flow will be dropped. The JMS consumer will not acknowledge the message but instead
will send back a "negative acknowledgment," or NAK. As a result, the JMS Broker will redeliver the message later. The main benefit to this approach is that if the NiFi instance were shutdown
(for example, due to a hardware failure or rescheduling the NiFi instance to run elsewhere in a cloud environment),
another NiFi instance (or some other software entirely) could consume the JMS message and handle it. There is no need to ensure that the NiFi instance
is able to restart.
Additionally, and related to the notion of failures, is the notion of timeouts when using the Stateless Engine. One of the configuration options for a Process Group is the "Stateless Flow Timeout."
This allows us to ensure that the data is processed within some expected amount of time and if not rollback the entire transaction. By default, the timeout is set to 1 minute. Any data that is
brought into the Stateless flow must finish processing within 1 minute, or a timeout will occur, and the entire transaction will be rolled back. The Traditional Engine has no notion of timeouts
for a dataflow.
We must also consider the transactionality that is used whenever a Stateless flow starts with an Input Port. We might, for example, consume data from any number of sources with a Process Group
that uses the Traditional Engine. Then, we might process the data using a Stateless flow. In this case, if any data is routed to a Failure Port, the entire transaction is rolled back, and the
original input FlowFile is instead transferred to the Failure Port - NOT the FlowFile that triggered the failure. If a timeout occurs, the original FlowFile is penalized and transferred back
to its original queue so that it can be processed again.
==== Data Durability
A key benefit of NiFi is its data durability. While many ETL products and data movement offerings provide in-memory processing, NiFi instead provides data durability. All data is persisted
to its underlying repositories. Upon restart, NiFi continues processing data where it left off. This means that it is capable of handling virtually any protocol reliably, even protocols
where the receiver does not send back application-level acknowledgments, such as direct TCP connections.
However, while this is true of the Traditional Engine, it is not true of the Stateless Engine. In contrast, the Stateless Engine, by design, drops data upon restart. This means that the
Stateless Engine is not a safe choice if data must be persisted by NiFi. However, for protocols that provide application-level acknowledgments such as JMS, Apache Kafka, Amazon Kinesis, etc.
it can be a great choice. Additionally, for protocols such as HTTP, NiFi offers processors that are capable of receiving data, performing some processing, and then sending an acknowledgment.
As such, it is safe even when NiFi is responsible for accepting incoming connections. The key here is the application-level acknowledgment message that is sent from NiFi.
==== Data Ordering
NiFi offers the ability to order data in each queue by use of FlowFile Prioritizers. This allows the dataflow developer to determine that specific data is the most important
and ensure that it is processed and/or sent to its destination first. This has proven to be particularly powerful at locations that have less reliable communication mediums,
in which communications channels may be down intermittently and when up need to be taken advantage of to send the most important data first.
The Stateless Engine, however, is designed to process data with a higher degree of transactionality and does not generally buffer large amounts of data while the destination
is unreachable. As such, it ignores the configured Prioritizers and always processes data using a First In First Out (FIFO) queue.
==== Data Provenance
Under normal circumstances, the Data Provenance that is produced by flows using the Stateless Engine are persisted to the same underlying Provenance Repository as the
Traditional Engine and as such the Provenance data appears in the same way. However, because the Stateless Engine may roll back transactions on failure, we do to consider
how this plays into the role of Data Provenance. Otherwise, we might show that the content was modified, then the attribute modified, and the data cloned - but upon restart
the data is would still be queued up outside of the Stateless Engine, and this would lead us to believe that the FlowFile's content were different than they truly are.
As such, the Provenance Events are not stored into the Provenance Repository until the transaction completes for a Stateless flow. If a FlowFile is routed to a Failure Port,
or if the invocation times out, the Provenance Events are discard. There are, however, two exceptions to this rule: `SEND` and `REMOTE_INVOCATION` events. Even if the transaction is
rolled back, the fact that data was sent, or that some remote invocation occurred cannot be rolled back. Therefore, the Provenance Repository is still updated to note the fact that
these events occurred.
==== Site-to-Site
NiFi's site-to-site mechanism provides a way to send FlowFiles, including their attributes from one NiFi instance to another in an efficient, reliable, and secure manner.
This requires that one NiFi instance use a Remote Process Group, which acts as a client and establishes the socket connection to the other NiFi instance. When using the
Traditional Engine, either NiFi instance can act as the client or the server. That is, it can make use of both the Remote Process Group, as well as Input Ports that allow for
remote connections. When using the Stateless Engine, the NiFi instance may make use of the Remote Process Group and act as a client to either push or pull data. However, it cannot
make use of Input Ports that allow remote connections and therefore cannot act as the server.
==== Processor Internal State
When the Stateless Engine is used and the number of Concurrent Tasks is set to a value greater than one, each Concurrent Task is run with a different copy of each Processor.
Because of this, Processors that require keeping internal state will not share that state between copies of the flow. For example, if a flow has a MergeContent processor
that must accumulate 1,000 FlowFiles before merging the data, and the number of Concurrent Tasks is set to 3, each concurrent task may have up to 999 FlowFiles, for a total
of 2,997 FlowFiles without the data being merged. This is important to keep in mind if designing flows where the Processor requires internal state.
Additionally, because of this, any flow that consists of a Processor scheduled to run on the Primary Node Only is limited to use a single Concurrent Task. Setting the value
higher than one will result in a warning being logged when started and a value of one will be used, regardless of the configured value.
=== Stateless Flow Failure Handling
Handling failures with the Traditional Engine is straight forward. Data is routed to a `failure` relationship, for example, and is dealt with accordingly.
While this is straight forward, it is not always the desired outcome, however.
The Stateless Engine approaches failure handling very differently.
==== Failure Ports
The Stateless Engine allows configuring an Output Port as a "Failure Port" meaning that if any FlowFile reaches the Output Port, the entire transaction is
failed. This Output Port can live at any level within the Process Group hierarchy.
If this occurs, the entire transaction is rolled back. That is, any FlowFiles that are currently processing within that transaction will be discarded.
If the source of the transaction is an Input Port, the FlowFile that was brought in via that Input Port will be transferred to the given Failure Output Port.
If the source of the transaction was a Processor, that Processor will be given the opportunity to send a negative acknowledgment ("NAK") or cleanup any resources,
as necessary.
==== Timeouts
If a Stateless flow does not complete within the configured timeout period, the transaction is also rolled back, in much the same way as when a FlowFile is routed
to a Failure Port. However, there is one very notable difference. Whereas a Failure Port indicates where to route the incoming FlowFile, a timeout has no such
output denoted. Instead, the input FlowFile, if one exists, is penalized and added back to its input queue so that it can be tried again after some delay.
=== Stateless Flow Design Considerations
[[single_source_single_destination]]
==== Single Source, Single Destination
Because a Stateless transaction may be rolled back, it is generally a best practice to have a flow consist of a single source and a single destination.
Consider the case when a FlowFile is received from a single source. It is then to be sent to three different destinations. After being successfully sent to two
of the three destinations, it fails to be sent to third destination because the destination is down for periodic maintenance. As a result, the transaction is rolled back.
The data is then pulled a second time. It is again sent to first two destinations successfully and again fails to be pushed to the third destination. It is again rolled back.
This may happen many times, and each time the data is sent to the first two destinations. Or perhaps sometimes it is sent only to the first destination and the second fails.
In such a situation, the flow is very inefficient at best, when the first two destinations are idempotent. At worst, the first and second destinations are not idempotent, and
there is a very significant, uncontrolled amount of data duplication.
This problem is further compounded if the flow is made up of multiple destinations. Consider a flow, for instance, that has two Processors, each responsible for consuming data from
a different source. That data is then merged together into a single FlowFile, which is then pushed to the three destinations. Now, if data is rolled back the source Processors may pull
different data from each of the sources. As a result, the data published cannot be idempotent, as it is made up of different data each time, and the pairing of data from each of those sources
is different each time.
As a result, it is recommended to keep Stateless flows to a single source and a single destination. At a minimum, the destinations other than the last should be idempotent.
==== Use Caution When Merging
Because each concurrent task runs a separate copy of the Stateless flow, it is important to keep in mind that the minimum number of incoming FlowFiles needed for a merge (via MergeContent
or MergeRecord, for example) may increase by a multiple of the number of concurrent tasks. As such, it may take a significant amount of time for that that much data to queue up.
Additionally, the Stateless flow has a timeout to limit how long it can run. Because of this, it is important to use caution when configuring limits for these types of Processors.
It is always advisable to set a Max Bin Age less than the timeout of the Stateless flow in order to ensure that the dataflow doesn't continually timeout. While it is true that continually
causing a timeout that rolls back the transaction should not prevent proper data delivery, it can cause unnecessary latency and use significant amounts of computing resources if transactions
are continually rolled back.
As a result, it is recommended that if using Processors that merge together multiple FlowFiles that the minimum limits are kept small enough that transactions will not continually timeout,
or that a Max Bin Age, or something equivalent, is configured for such Processors so that they may successfully continue even when large amounts of data are not available. It is also worth
noting that if larger output FlowFiles are desirable, using a lower number of Concurrent Tasks is more likely to produce the larger outputs, since the incoming data will be split across
fewer copies of the flow.
[[when_to_use_stateless_standard]]
=== When to Use Stateless vs Standard Execution Engine
We have, thus far, discussed the difference between the two Execution Engines. It is important to round out the conversation by highlighting when it will make sense to use one
Execution Engine or the other.
Typically, the Stateless Engine may be preferred when three conditions are met: the source is transactional, there is one destination, and
there is a desire to treat the processing of a given FlowFile across multiple Processors as a single transaction.
Firstly, the source should be transactional. This means that NiFi must have some mechanism for indicating that it has received the data, and the source of the data must have
a mechanism for re-sending the data if it doesn't receive that acknowledgment. This is the case for most queuing protocols, such as JMS, Kafka, Amazon SQS, and Kinesis. It is also
the case for HTTP requests or RPC types of requests. However, it is not the case for lower-level protocols such as direct TCP connections that have no higher-level protocol on top.
Secondly, the should be one destination. As discussed in the <<single_source_single_destination>> section above, a Stateless Flow should generally have a single source and a single destination.
And thirdly it should be desirable to treat the entire flow as a single transaction. If any part of an incoming FlowFile fails processing, the entire transaction fails. If this is not
the desired behavior, the Traditional Engine should instead be used.
Generally speaking, the Traditional Engine will provide somewhat better performance, simply because it allows for queuing larger numbers of FlowFiles between Processors and allocating more resources
to individual Processors. Additionally, if a destination is down, the Traditional Engine allows the data to be queued up and periodically retried until the destination becomes available
again. All of the transformation and processing needed to get the data into the state where it can be sent does not need to be performed more than once, because NiFi queues the data and restores
it at the appropriate step in the flow. Additionally, if a single FlowFile must be delivered to multiple destinations, the Traditional Engine makes this trivial while the Stateless Engine
complicates matters.
While other considerations will likely need to be taken into account for each situation, these are often the most important concepts to consider.
== Command and Control of the DataFlow

View File

@ -36,7 +36,7 @@ The NiFi Kafka Connector allows users to do just that!
When a dataflow is to be run within Kafka Connect, it is run using the Stateless NiFi dataflow engine. For more information, see the README of the
Stateless NiFi module.
Stateless NiFi differs from the traditional NiFi engine in a few ways. For one, Stateless NiFi is an engine that is designed to be embedded. This makes it
Stateless NiFi differs from the standard NiFi engine in a few ways. For one, Stateless NiFi is an engine that is designed to be embedded. This makes it
very convenient to run within the Kafka Connect framework.
Stateless NiFi does not provide a user interface (UI) or a REST API and does not support modifying the dataflow while it is running.
@ -47,7 +47,7 @@ data is expected to be routed to exactly one of them, such as a 'Failure' destin
Stateless NiFi does not currently provide access to data lineage/provenance.
Stateless NiFi does not support cyclic graphs. While it is common and desirable in traditional NiFi to have a 'failure' relationship from a Processor
Stateless NiFi does not support cyclic graphs. While it is common and desirable in standard NiFi deployments to have a 'failure' relationship from a Processor
route back to the same processor, this can result in a StackOverflowException in Stateless NiFi. The preferred approach in Stateless NiFi is to create
an Output Port for failures and route the data to that Output Port.
@ -56,7 +56,7 @@ an Output Port for failures and route the data to that Output Port.
NiFi supports two different Kafka Connectors: a source connector and a sink connector. Each of these have much of the same configuration
elements but do have some differences. The configuration for each of these connectors is described below.
In order to build a flow that will be run in Kafka Connect, the dataflow must first be built. This is accomplished using a traditional deployment
In order to build a flow that will be run in Kafka Connect, the dataflow must first be built. This is accomplished using a standard deployment
of NiFi. So, while NiFi does not have to be deployed in a production environment in order to use the Kafka Connector, it is necessary in a development
environment for building the actual dataflow that is to be deployed.
@ -137,8 +137,8 @@ will be used as the Kafka message key.
recommended to use the `value.converter` of `org.apache.kafka.connect.converters.ByteArrayConverter` as NiFi already has the data serialized as a byte array
and is very adept at formatting the data as it needs to be.
`(4) tasks.max`: The maximum number of tasks/threads to use to run the dataflow. Unlike traditional NiFi, with Stateless NiFi, the entire dataflow is run from start
to finish with a single thread. However, multiple threads can be used to run multiple copies of the dataflow.
`(4) tasks.max`: The maximum number of tasks/threads to use to run the dataflow. Unlike the standard NiFi engine, with Stateless NiFi, the entire dataflow is run from start
to finish with a single thread. However, multiple threads can be used to run multiple copies of the dataflow, each with their own data.
`(5) name`: The name of the connect instance. This should match the first `name` field.
@ -199,9 +199,9 @@ connector lives in. Because this directory may not be writable, and to aid in up
### Transactional sources
Unlike with traditional NiFi, Stateless NiFi keeps the contents of FlowFiles in memory. It is important to understand that as long as the source of the data is
Unlike with standard NiFi deployments, Stateless NiFi keeps the contents of FlowFiles in memory. It is important to understand that as long as the source of the data is
replayable and transactional, there is no concern over data loss. This is handled by treating the entire dataflow as a single transaction. Once data is obtained
from some source component, the data is transferred to the next processor in the flow. At that point, in traditional NiFi, the processor would acknowledge the data
from some source component, the data is transferred to the next processor in the flow. At that point, in a standard NiFi deployment, the processor would acknowledge the data
and NiFi would take ownership of that data, having persisted it to disk.
With Stateless NiFi, however, the data will not yet be acknowledged. Instead, the data will be transferred to the next processor in the flow, and it will perform
@ -277,7 +277,7 @@ will be used as the Kafka message key.
recommended to use the `value.converter` of `org.apache.kafka.connect.converters.ByteArrayConverter` as NiFi already has the data serialized as a byte array
and is very adept at formatting the data as it needs to be.
`(4) tasks.max`: The maximum number of tasks/threads to use to run the dataflow. Unlike traditional NiFi, with Stateless NiFi, the entire dataflow is run from start
`(4) tasks.max`: The maximum number of tasks/threads to use to run the dataflow. Unlike standard NiFi deployments, with Stateless NiFi, the entire dataflow is run from start
to finish with a single thread. However, multiple threads can be used to run multiple copies of the dataflow.
`(5) name`: The name of the connect instance. This should match the first `name` field.
@ -342,7 +342,7 @@ can often be delivered directly to a sink. For example, a PublishJMS processor i
sent to S3 or to HDFS, those services will perform much better if the data is first batched, or merged, together. The MergeContent and MergeRecord
processors are extremely popular in NiFi for this reason. They allow many small FlowFiles to be merged together into one larger FlowFile.
With traditional NiFi, we can simply set a minimum and maximum size for the merged data along with a timeout. However, with Stateless NiFi and Kafka Connect,
With the standard NiFi engine, we can simply set a minimum and maximum size for the merged data along with a timeout. However, with Stateless NiFi and Kafka Connect,
this may not work as well, because only a limited number of FlowFiles will be made available to the Processor. We can still use these Processor in
order to merge the data together, but with a bit of a limitation.

View File

@ -35,7 +35,11 @@
<version>2.6.3</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-api</artifactId>
<scope>runtime</scope>
</dependency>
<!-- Dependencies for integration tests. These must be excluded from the main assembly -->
<!-- TODO: Probably should separate this into separate integration test module -->

View File

@ -195,7 +195,7 @@ public class StatelessNiFiSourceTaskIT {
assertEquals(1, sourceOffset.size());
final String generateProcessorId = sourceOffset.keySet().iterator().next();
final String serializedStateMap = "{\"version\":0,\"stateValues\":{\"count\":\"1\"}}";
final String serializedStateMap = "{\"version\":\"1\",\"stateValues\":{\"count\":\"1\"}}";
final Map<String, ?> expectedSourceOffset = Collections.singletonMap(generateProcessorId, serializedStateMap);
assertEquals(expectedSourceOffset, sourceOffset);

View File

@ -0,0 +1,31 @@
/*
* 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.diagnostics;
public enum DiagnosticLevel {
/**
* Only basic information should be gathered.
*/
BASIC,
/**
* Detailed diagnostics information should be gathered, even if it is more expensive and/or generates verbose output.
*/
VERBOSE;
}

View File

@ -16,8 +16,13 @@
*/
package org.apache.nifi.diagnostics;
import org.apache.nifi.controller.repository.claim.ResourceClaim;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.Map;
import java.util.Set;
/**
* Diagnostics for the JVM.
@ -49,6 +54,8 @@ public class SystemDiagnostics implements Cloneable {
private Map<String, StorageUsage> contentRepositoryStorageUsage;
private Map<String, StorageUsage> provenanceRepositoryStorageUsage;
private Map<String, GarbageCollection> garbageCollection;
private Map<ResourceClaim, Integer> claimantCounts;
private Set<ResourceClaim> destructableClaims;
private long creationTimestamp;
@ -228,6 +235,22 @@ public class SystemDiagnostics implements Cloneable {
this.openFileHandles = openFileHandles;
}
public Map<ResourceClaim, Integer> getClaimantCounts() {
return claimantCounts;
}
public void setClaimantCounts(final Map<ResourceClaim, Integer> claimantCounts) {
this.claimantCounts = claimantCounts;
}
public Set<ResourceClaim> getDestructableClaims() {
return destructableClaims;
}
public void setDestructableClaims(final Set<ResourceClaim> destructableClaims) {
this.destructableClaims = destructableClaims;
}
@Override
public SystemDiagnostics clone() {
final SystemDiagnostics clonedObj = new SystemDiagnostics();
@ -270,6 +293,8 @@ public class SystemDiagnostics implements Cloneable {
clonedObj.totalPhysicalMemory = totalPhysicalMemory;
clonedObj.openFileHandles = openFileHandles;
clonedObj.maxOpenFileHandles = maxOpenFileHandles;
clonedObj.claimantCounts = this.claimantCounts == null ? null : new HashMap<>(this.claimantCounts);
clonedObj.destructableClaims = this.destructableClaims == null ? null : new HashSet<>(this.destructableClaims);
return clonedObj;
}

View File

@ -30,6 +30,7 @@ public class AffectedComponentDTO {
public static final String COMPONENT_TYPE_OUTPUT_PORT = "OUTPUT_PORT";
public static final String COMPONENT_TYPE_REMOTE_INPUT_PORT = "REMOTE_INPUT_PORT";
public static final String COMPONENT_TYPE_REMOTE_OUTPUT_PORT = "REMOTE_OUTPUT_PORT";
public static final String COMPONENT_TYPE_STATELESS_GROUP = "STATELESS_GROUP";
private String processGroupId;
private String id;
@ -61,7 +62,7 @@ public class AffectedComponentDTO {
@ApiModelProperty(value = "The type of this component",
allowableValues = COMPONENT_TYPE_PROCESSOR + "," + COMPONENT_TYPE_CONTROLLER_SERVICE + ", "
+ COMPONENT_TYPE_INPUT_PORT + ", " + COMPONENT_TYPE_OUTPUT_PORT + ", "
+ COMPONENT_TYPE_REMOTE_INPUT_PORT + ", " + COMPONENT_TYPE_REMOTE_OUTPUT_PORT)
+ COMPONENT_TYPE_REMOTE_INPUT_PORT + ", " + COMPONENT_TYPE_REMOTE_OUTPUT_PORT + ", " + COMPONENT_TYPE_STATELESS_GROUP)
public String getReferenceType() {
return referenceType;
}

View File

@ -37,6 +37,7 @@ public class PortDTO extends ComponentDTO {
private Set<String> userAccessControl;
private Set<String> groupAccessControl;
private Boolean allowRemoteAccess;
private String portFunction;
private Collection<String> validationErrors;
@ -186,4 +187,15 @@ public class PortDTO extends ComponentDTO {
public void setAllowRemoteAccess(Boolean allowRemoteAccess) {
this.allowRemoteAccess = allowRemoteAccess;
}
@ApiModelProperty(value = "Specifies how the Port functions",
allowableValues = "STANDARD, FAILURE"
)
public String getPortFunction() {
return portFunction;
}
public void setPortFunction(final String portFunction) {
this.portFunction = portFunction;
}
}

View File

@ -40,6 +40,9 @@ public class ProcessGroupDTO extends ComponentDTO {
private Long defaultBackPressureObjectThreshold;
private String defaultBackPressureDataSizeThreshold;
private String logFileSuffix;
private String executionEngine;
private Integer maxConcurrentTasks;
private String statelessFlowTimeout;
private Integer runningCount;
private Integer stoppedCount;
@ -60,6 +63,8 @@ public class ProcessGroupDTO extends ComponentDTO {
private Integer publicInputPortCount;
private Integer publicOutputPortCount;
private String statelessGroupScheduledState;
private FlowSnippetDTO contents;
public ProcessGroupDTO() {
@ -413,4 +418,42 @@ public class ProcessGroupDTO extends ComponentDTO {
public void setLogFileSuffix(final String logFileSuffix) {
this.logFileSuffix = logFileSuffix;
}
@ApiModelProperty(value = "The Execution Engine that should be used to run the flow represented by this Process Group.",
allowableValues = "STATELESS, STANDARD, INHERITED")
public String getExecutionEngine() {
return executionEngine;
}
public void setExecutionEngine(final String executionEngine) {
this.executionEngine = executionEngine;
}
@ApiModelProperty(value = "If the Process Group is configured to run in using the Stateless Engine, represents the current state. Otherwise, will be STOPPED.",
allowableValues = "STOPPED, RUNNING")
public String getStatelessGroupScheduledState() {
return statelessGroupScheduledState;
}
public void setStatelessGroupScheduledState(final String state) {
this.statelessGroupScheduledState = state;
}
@ApiModelProperty("The maximum number of concurrent tasks to use when running the flow using the Stateless Engine")
public Integer getMaxConcurrentTasks() {
return maxConcurrentTasks;
}
public void setMaxConcurrentTasks(final Integer maxConcurrentTasks) {
this.maxConcurrentTasks = maxConcurrentTasks;
}
@ApiModelProperty("The maximum amount of time that the flow can be run using the Stateless Engine before the flow times out")
public String getStatelessFlowTimeout() {
return statelessFlowTimeout;
}
public void setStatelessFlowTimeout(final String timeout) {
this.statelessFlowTimeout = timeout;
}
}

View File

@ -24,6 +24,7 @@ import javax.xml.bind.annotation.XmlType;
import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
import java.util.Date;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Set;
/**
@ -64,6 +65,7 @@ public class SystemDiagnosticsSnapshotDTO implements Cloneable {
private Set<StorageUsageDTO> contentRepositoryStorageUsage;
private Set<StorageUsageDTO> provenanceRepositoryStorageUsage;
private Set<GarbageCollectionDTO> garbageCollection;
private List<ResourceClaimDetailsDTO> resourceClaimDetails;
private Date statsLastRefreshed;
@ -232,6 +234,14 @@ public class SystemDiagnosticsSnapshotDTO implements Cloneable {
this.garbageCollection = garbageCollection;
}
public List<ResourceClaimDetailsDTO> getResourceClaimDetails() {
return resourceClaimDetails;
}
public void setResourceClaimDetails(final List<ResourceClaimDetailsDTO> resourceClaimDetails) {
this.resourceClaimDetails = resourceClaimDetails;
}
@XmlJavaTypeAdapter(TimeAdapter.class)
@ApiModelProperty(
value = "When the diagnostics were generated.",
@ -362,6 +372,7 @@ public class SystemDiagnosticsSnapshotDTO implements Cloneable {
other.setUsedHeapBytes(getUsedHeapBytes());
other.setUsedNonHeap(getUsedNonHeap());
other.setUsedNonHeapBytes(getUsedNonHeapBytes());
other.setResourceClaimDetails(getResourceClaimDetails());
other.setFlowFileRepositoryStorageUsage(getFlowFileRepositoryStorageUsage().clone());
@ -538,6 +549,93 @@ public class SystemDiagnosticsSnapshotDTO implements Cloneable {
}
}
@XmlType(name = "resourceClaimDetails")
public static class ResourceClaimDetailsDTO implements Cloneable {
private String container;
private String section;
private String identifier;
private Boolean inUse;
private Boolean awaitingDestruction;
private Boolean writable;
private Integer claimantCount;
@ApiModelProperty("The container of the Content Repository in which the Resource Claim exists")
public String getContainer() {
return container;
}
public void setContainer(final String container) {
this.container = container;
}
@ApiModelProperty("The section of the Content Repository in which the Resource Claim exists")
public String getSection() {
return section;
}
public void setSection(final String section) {
this.section = section;
}
@ApiModelProperty("The identifier of the Resource Claim")
public String getIdentifier() {
return identifier;
}
public void setIdentifier(final String identifier) {
this.identifier = identifier;
}
@ApiModelProperty("Whether or not the Resource Claim is in use")
public Boolean getInUse() {
return inUse;
}
public void setInUse(final Boolean inUse) {
this.inUse = inUse;
}
@ApiModelProperty("Whether or not the Resource Claim is awaiting destruction")
public Boolean getAwaitingDestruction() {
return awaitingDestruction;
}
public void setAwaitingDestruction(final Boolean awaitingDestruction) {
this.awaitingDestruction = awaitingDestruction;
}
@ApiModelProperty("The number of FlowFiles that have a claim to the Resource")
public Integer getClaimantCount() {
return claimantCount;
}
public void setClaimantCount(final Integer claimantCount) {
this.claimantCount = claimantCount;
}
@ApiModelProperty("Whether or not the Resource Claim can still have more data written to it")
public Boolean getWritable() {
return writable;
}
public void setWritable(final Boolean writable) {
this.writable = writable;
}
public ResourceClaimDetailsDTO clone() {
final ResourceClaimDetailsDTO other = new ResourceClaimDetailsDTO();
other.setContainer(getContainer());
other.setSection(getSection());
other.setIdentifier(getIdentifier());
other.setInUse(getInUse());
other.setAwaitingDestruction(getAwaitingDestruction());
other.setClaimantCount(getClaimantCount());
other.setWritable(getWritable());
return other;
}
}
/**
* Details for garbage collection.
*/

View File

@ -17,6 +17,7 @@
package org.apache.nifi.web.api.dto.status;
import io.swagger.annotations.ApiModelProperty;
import io.swagger.annotations.ApiModelProperty.AccessMode;
import org.apache.nifi.web.api.entity.ConnectionStatusSnapshotEntity;
import org.apache.nifi.web.api.entity.PortStatusSnapshotEntity;
import org.apache.nifi.web.api.entity.ProcessGroupStatusSnapshotEntity;
@ -44,6 +45,7 @@ public class ProcessGroupStatusSnapshotDTO implements Cloneable {
private Collection<PortStatusSnapshotEntity> outputPortStatusSnapshots;
private String versionedFlowState;
private int statelessActiveThreadCount;
private Integer flowFilesIn = 0;
private Long bytesIn = 0L;
@ -117,6 +119,16 @@ public class ProcessGroupStatusSnapshotDTO implements Cloneable {
this.versionedFlowState = versionedFlowState;
}
@ApiModelProperty(accessMode = AccessMode.READ_ONLY,
value = "The current number of active threads for the Process Group, when running in Stateless mode.")
public int getStatelessActiveThreadCount() {
return statelessActiveThreadCount;
}
public void setStatelessActiveThreadCount(int statelessActiveThreadCount) {
this.statelessActiveThreadCount = statelessActiveThreadCount;
}
/**
* @return active thread count for this process group
*/
@ -513,6 +525,7 @@ public class ProcessGroupStatusSnapshotDTO implements Cloneable {
other.setId(getId());
other.setName(getName());
other.setVersionedFlowState(getVersionedFlowState());
other.setStatelessActiveThreadCount(getStatelessActiveThreadCount());
other.setBytesIn(getBytesIn());
other.setFlowFilesIn(getFlowFilesIn());

View File

@ -17,6 +17,7 @@
package org.apache.nifi.cluster.manager;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.groups.StatelessGroupScheduledState;
import org.apache.nifi.web.api.dto.ProcessGroupDTO;
import org.apache.nifi.web.api.dto.VersionControlInformationDTO;
import org.apache.nifi.web.api.dto.status.ProcessGroupStatusDTO;
@ -61,11 +62,45 @@ public class ProcessGroupEntityMerger implements ComponentEntityMerger<ProcessGr
final ProcessGroupDTO dto = entry.getValue();
final ParameterContextReferenceEntity parameterContextReferenceEntity = dto.getParameterContext();
clientDto.setStatelessGroupScheduledState(mergeScheduledState(clientDto.getStatelessGroupScheduledState(), dto.getStatelessGroupScheduledState()));
PermissionsDtoMerger.mergePermissions(clientParameterContextEntity.getPermissions(), parameterContextReferenceEntity.getPermissions());
}
}
}
private static String mergeScheduledState(final String stateAName, final String stateBName) {
final StatelessGroupScheduledState stateA = getScheduledState(stateAName);
final StatelessGroupScheduledState stateB = getScheduledState(stateBName);
if (stateA == null && stateB == null) {
return null;
}
if (stateA == null) {
return stateB.name();
}
if (stateB == null) {
return stateA.name();
}
if (stateA == StatelessGroupScheduledState.RUNNING || stateB == StatelessGroupScheduledState.RUNNING) {
return StatelessGroupScheduledState.RUNNING.name();
}
return StatelessGroupScheduledState.STOPPED.name();
}
private static StatelessGroupScheduledState getScheduledState(final String value) {
if (value == null) {
return null;
}
try {
return StatelessGroupScheduledState.valueOf(value);
} catch (final Exception e) {
return null;
}
}
@Override
public void mergeStatus(ProcessGroupStatusDTO clientStatus, boolean clientStatusReadablePermission, ProcessGroupStatusDTO status, boolean statusReadablePermission,
NodeIdentifier statusNodeIdentifier) {

View File

@ -149,6 +149,8 @@ public class StatusMerger {
target.setVersionedFlowState(VersionedFlowState.SYNC_FAILURE.name());
}
target.setStatelessActiveThreadCount(target.getStatelessActiveThreadCount() + toMerge.getStatelessActiveThreadCount());
target.setBytesIn(target.getBytesIn() + toMerge.getBytesIn());
target.setFlowFilesIn(target.getFlowFilesIn() + toMerge.getFlowFilesIn());
@ -705,6 +707,13 @@ public class StatusMerger {
merge(target.getFlowFileRepositoryStorageUsage(), toMerge.getFlowFileRepositoryStorageUsage());
mergeGarbageCollection(target.getGarbageCollection(), toMerge.getGarbageCollection());
if (target.getResourceClaimDetails() == null && toMerge.getResourceClaimDetails() != null) {
target.setResourceClaimDetails(new ArrayList<>());
}
if (toMerge.getResourceClaimDetails() != null) {
target.getResourceClaimDetails().addAll(toMerge.getResourceClaimDetails());
}
updatePrettyPrintedFields(target);
}

View File

@ -96,6 +96,12 @@
<artifactId>nifi-registry-flow-diff</artifactId>
<version>2.0.0-SNAPSHOT</version>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-stateless-api</artifactId>
<version>2.0.0-SNAPSHOT</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-mock</artifactId>

View File

@ -0,0 +1,145 @@
/*
* 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.connectable;
import org.apache.nifi.controller.queue.FlowFileQueue;
import org.apache.nifi.controller.repository.ContentRepository;
import org.apache.nifi.controller.repository.FlowFileRecord;
import org.apache.nifi.controller.repository.FlowFileRepository;
import org.apache.nifi.controller.repository.RepositoryRecord;
import org.apache.nifi.controller.repository.StandardFlowFileRecord;
import org.apache.nifi.controller.repository.StandardRepositoryRecord;
import org.apache.nifi.controller.repository.claim.ContentClaim;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.UUID;
public class ConnectionUtils {
/**
* Creates the FlowFileRecords and the RepositoryRecords necessary for transferring the given FlowFile to all of the given destinations.
* If only a single destination is given, the returned result will contain only the original FlowFile and a RepositoryRecord that represents that
* FlowFile. If multiple destinations are given, the returned result will contain the original FlowFile, as well as a cloned FlowFile for each of
* the destinations. If one or more clones are created, and a Content Repository is provided, the Claimant Count will be incremented for each clone
* (but not the original FlowFile). If the Content Repository is <code>null</code>, the Claimant Counts will not be incremented.
*
* @param flowFile the FlowFile to clone as necessary
* @param destinations the destinations that the FlowFile will be transferred to
* @param flowFileRepository the FlowFile Repository to use to generate the FlowFile ID for each clone
* @param contentRepository the Content Repository to use to increment the Claimant Count for each clone, or <code>null</code> if the caller will
* take on the responsibility of incrementing the Claimant Counts
* @return a FlowFileCloneResult that contains the FlowFiles to enqueue and the RepositoryRecords to create
*/
public static FlowFileCloneResult clone(final FlowFileRecord flowFile, final Collection<Connection> destinations,
final FlowFileRepository flowFileRepository, final ContentRepository contentRepository) {
final Map<FlowFileQueue, List<FlowFileRecord>> flowFilesToEnqueue = new HashMap<>();
// If only a single destination, we can simply add the FlowFile to the destination.
if (destinations.size() == 1) {
final Connection firstConnection = destinations.iterator().next();
flowFilesToEnqueue.put(firstConnection.getFlowFileQueue(), Collections.singletonList(flowFile));
final RepositoryRecord repoRecord = createRepositoryRecord(flowFile, firstConnection.getFlowFileQueue());
return new FlowFileCloneResult(flowFilesToEnqueue, Collections.singletonList(repoRecord));
}
final List<RepositoryRecord> repositoryRecords = new ArrayList<>();
final Iterator<Connection> itr = destinations.iterator();
final Connection firstConnection = itr.next();
// Clone the FlowFile to all other destinations.
while (itr.hasNext()) {
final Connection destination = itr.next();
final StandardFlowFileRecord.Builder builder = new StandardFlowFileRecord.Builder().fromFlowFile(flowFile);
final long id = flowFileRepository.getNextFlowFileSequence();
builder.id(id);
final String newUuid = UUID.randomUUID().toString();
builder.addAttribute(CoreAttributes.UUID.key(), newUuid);
final FlowFileRecord clone = builder.build();
final ContentClaim claim = clone.getContentClaim();
if (claim != null && contentRepository != null) {
contentRepository.incrementClaimaintCount(claim);
}
final RepositoryRecord repoRecord = createRepositoryRecord(clone, destination.getFlowFileQueue());
repositoryRecords.add(repoRecord);
final List<FlowFileRecord> flowFiles = flowFilesToEnqueue.computeIfAbsent(destination.getFlowFileQueue(), k -> new ArrayList<>());
flowFiles.add(clone);
}
// Enqueue the FlowFile into the first connection
flowFilesToEnqueue.put(firstConnection.getFlowFileQueue(), Collections.singletonList(flowFile));
repositoryRecords.add(createRepositoryRecord(flowFile, firstConnection.getFlowFileQueue()));
return new FlowFileCloneResult(flowFilesToEnqueue, repositoryRecords);
}
private static RepositoryRecord createRepositoryRecord(final FlowFileRecord flowFile, final FlowFileQueue destinationQueue) {
final StandardRepositoryRecord repoRecord = new StandardRepositoryRecord(null, flowFile);
repoRecord.setWorking(flowFile, Collections.emptyMap(), false);
repoRecord.setDestination(destinationQueue);
return repoRecord;
}
public static class FlowFileCloneResult {
private final Map<FlowFileQueue, List<FlowFileRecord>> flowFilesToEnqueue;
private final List<RepositoryRecord> repositoryRecords;
private FlowFileCloneResult(final Map<FlowFileQueue, List<FlowFileRecord>> flowFilesToEnqueue, final List<RepositoryRecord> repositoryRecords) {
this.flowFilesToEnqueue = flowFilesToEnqueue;
this.repositoryRecords = repositoryRecords;
}
public List<RepositoryRecord> getRepositoryRecords() {
return repositoryRecords;
}
public int distributeFlowFiles() {
if (flowFilesToEnqueue.isEmpty()) {
return 0;
}
int enqueued = 0;
for (final Map.Entry<FlowFileQueue, List<FlowFileRecord>> entry : flowFilesToEnqueue.entrySet()) {
final FlowFileQueue queue = entry.getKey();
final List<FlowFileRecord> flowFiles = entry.getValue();
if (!flowFiles.isEmpty()) {
queue.putAll(flowFiles);
enqueued += flowFiles.size();
}
}
return enqueued;
}
public Map<FlowFileQueue, List<FlowFileRecord>> getFlowFilesToEnqueue() {
return flowFilesToEnqueue;
}
}
}

View File

@ -16,9 +16,11 @@
*/
package org.apache.nifi.connectable;
import org.apache.nifi.components.PortFunction;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.controller.AbstractPort;
import org.apache.nifi.controller.ProcessScheduler;
import org.apache.nifi.flow.ExecutionEngine;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.groups.DataValve;
import org.apache.nifi.groups.FlowFileConcurrency;
@ -50,14 +52,6 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
public class LocalPort extends AbstractPort {
private static final Logger logger = LoggerFactory.getLogger(LocalPort.class);
// "_nifi.funnel.max.concurrent.tasks" is an experimental NiFi property allowing users to configure
// the number of concurrent tasks to schedule for local ports and funnels.
static final String MAX_CONCURRENT_TASKS_PROP_NAME = "_nifi.funnel.max.concurrent.tasks";
// "_nifi.funnel.max.transferred.flowfiles" is an experimental NiFi property allowing users to configure
// the maximum number of FlowFiles transferred each time a funnel or local port runs (rounded up to the nearest 1000).
static final String MAX_TRANSFERRED_FLOWFILES_PROP_NAME = "_nifi.funnel.max.transferred.flowfiles";
private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
private final Lock readLock = rwLock.readLock();
private final Lock writeLock = rwLock.writeLock();
@ -73,19 +67,52 @@ public class LocalPort extends AbstractPort {
setYieldPeriod(boredYieldDuration);
}
protected int getMaxIterations() {
return maxIterations;
}
@Override
public boolean isValid() {
return hasIncomingConnection() && hasOutboundConnection();
return hasIncomingConnection() && (!isOutboundConnectionRequired() || hasOutboundConnection());
}
private boolean hasOutboundConnection() {
return !getConnections(Relationship.ANONYMOUS).isEmpty();
}
/**
* <p>
* An Outbound Connection is required for a Local Port unless all of the following conditions are met:
* </p>
*
* <ul>
* <li>Group is using the Stateless Execution Engine</li>
* <li>There is no Input Port to the Group</li>
* <li>The Port is a failure port</li>
* </ul>
*
* <p>
* Under these conditions, it is not necessary to have an outbound connection because anything that gets routed to the
* Port will end up being rolled back anyway, and since there is no Input Port, there is no input FlowFile to route to failure.
* </p>
*
* @return true if an Outbound Connection is required in order for the Port to be valid, <code>false</code> otherwise
*/
private boolean isOutboundConnectionRequired() {
final ExecutionEngine engine = getProcessGroup().resolveExecutionEngine();
if (engine == ExecutionEngine.STANDARD) {
return true;
}
final PortFunction portFunction = getPortFunction();
if (portFunction != PortFunction.FAILURE) {
return true;
}
final boolean groupHasInputPort = !getProcessGroup().getInputPorts().isEmpty();
if (groupHasInputPort) {
return true;
}
return false;
}
@Override
public Collection<ValidationResult> getValidationErrors() {
final Collection<ValidationResult> validationErrors = new ArrayList<>();
@ -99,7 +126,7 @@ public class LocalPort extends AbstractPort {
}
// Outgoing connections are required but not set
if (!hasOutboundConnection()) {
if (!hasOutboundConnection() && isOutboundConnectionRequired()) {
validationErrors.add(new ValidationResult.Builder()
.explanation("Port has no outgoing connections")
.subject(String.format("Port '%s'", getName()))

View File

@ -16,7 +16,6 @@
*/
package org.apache.nifi.controller;
import org.apache.commons.lang3.builder.EqualsBuilder;
import org.apache.commons.lang3.builder.HashCodeBuilder;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.Restricted;
@ -134,7 +133,6 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
private static final Logger LOG = LoggerFactory.getLogger(StandardProcessorNode.class);
public static final String BULLETIN_OBSERVER_ID = "bulletin-observer";
public static final TimeUnit DEFAULT_TIME_UNIT = TimeUnit.MILLISECONDS;
public static final String DEFAULT_YIELD_PERIOD = "1 sec";
@ -143,7 +141,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
private final AtomicReference<ProcessGroup> processGroup;
private final AtomicReference<ProcessorDetails> processorRef;
private final AtomicReference<String> identifier;
private final String identifier;
private final Map<Connection, Connectable> destinations;
private final Map<Relationship, Set<Connection>> connections;
private final AtomicReference<Set<Relationship>> undefinedRelationshipsToTerminate;
@ -200,7 +198,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
final ProcessorDetails processorDetails = new ProcessorDetails(processor);
this.processorRef = new AtomicReference<>(processorDetails);
identifier = new AtomicReference<>(uuid);
identifier = uuid;
destinations = new ConcurrentHashMap<>();
connections = new ConcurrentHashMap<>();
incomingConnections = new AtomicReference<>(new ArrayList<>());
@ -345,7 +343,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
@Override
public String getIdentifier() {
return identifier.get();
return identifier;
}
/**
@ -679,7 +677,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
@Override
public synchronized void setBulletinLevel(final LogLevel level) {
LogRepositoryFactory.getRepository(getIdentifier()).setObservationLevel(BULLETIN_OBSERVER_ID, level);
LogRepositoryFactory.getRepository(getIdentifier()).setObservationLevel(level);
this.bulletinLevel = level;
}
@ -990,7 +988,8 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
@Override
public boolean isRunning() {
return getScheduledState().equals(ScheduledState.RUNNING) || hasActiveThreads;
final ScheduledState state = getScheduledState();
return state == ScheduledState.RUNNING || state == ScheduledState.STARTING || hasActiveThreads;
}
@Override
@ -1005,6 +1004,11 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
return false;
}
@Override
public Optional<ProcessGroup> getParentProcessGroup() {
return Optional.ofNullable(processGroup.get());
}
@Override
public int getActiveThreadCount() {
final int activeThreadCount = processScheduler.getActiveThreadCount(this);
@ -1287,8 +1291,9 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
if (!(other instanceof ProcessorNode)) {
return false;
}
final ProcessorNode on = (ProcessorNode) other;
return new EqualsBuilder().append(identifier.get(), on.getIdentifier()).isEquals();
return Objects.equals(identifier, on.getIdentifier());
}
@Override
@ -1390,29 +1395,12 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
@Override
public void verifyCanStart() {
this.verifyCanStart(null);
}
@Override
public void verifyCanStart(final Set<ControllerServiceNode> ignoredReferences) {
final ScheduledState currentState = getPhysicalScheduledState();
if (currentState != ScheduledState.STOPPED && currentState != ScheduledState.DISABLED) {
throw new IllegalStateException(this + " cannot be started because it is not stopped. Current state is " + currentState.name());
}
verifyNoActiveThreads();
switch (getValidationStatus()) {
case VALID:
return;
case VALIDATING:
throw new IllegalStateException(this + " cannot be started because its validation is still being performed");
}
final Collection<ValidationResult> validationErrors = getValidationErrors(ignoredReferences);
if (ignoredReferences != null && !validationErrors.isEmpty()) {
throw new IllegalStateException(this + " cannot be started because it is not currently valid");
}
}
@Override
@ -1527,13 +1515,14 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
*/
@Override
public void start(final ScheduledExecutorService taskScheduler, final long administrativeYieldMillis, final long timeoutMillis, final Supplier<ProcessContext> processContextFactory,
final SchedulingAgentCallback schedulingAgentCallback, final boolean failIfStopping) {
final SchedulingAgentCallback schedulingAgentCallback, final boolean failIfStopping, final boolean triggerLifecycleMethods) {
run(taskScheduler, administrativeYieldMillis, timeoutMillis, processContextFactory, schedulingAgentCallback, failIfStopping, ScheduledState.RUNNING, ScheduledState.STARTING);
run(taskScheduler, administrativeYieldMillis, timeoutMillis, processContextFactory, schedulingAgentCallback, failIfStopping,
ScheduledState.RUNNING, ScheduledState.STARTING, triggerLifecycleMethods);
}
/**
* Similar to {@link #start(ScheduledExecutorService, long, long, Supplier, SchedulingAgentCallback, boolean)}, except for the following:
* Similar to {@link #start(ScheduledExecutorService, long, long, Supplier, SchedulingAgentCallback, boolean, boolean)}, except for the following:
* <ul>
* <li>
* Once the {@link Processor#onTrigger(ProcessContext, ProcessSessionFactory)} method has been invoked successfully, the processor is scehduled to be stopped immediately.
@ -1549,11 +1538,12 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
public void runOnce(final ScheduledExecutorService taskScheduler, final long administrativeYieldMillis, final long timeoutMillis, final Supplier<ProcessContext> processContextFactory,
final SchedulingAgentCallback schedulingAgentCallback) {
run(taskScheduler, administrativeYieldMillis, timeoutMillis, processContextFactory, schedulingAgentCallback, true, ScheduledState.RUN_ONCE, ScheduledState.RUN_ONCE);
run(taskScheduler, administrativeYieldMillis, timeoutMillis, processContextFactory, schedulingAgentCallback, true, ScheduledState.RUN_ONCE, ScheduledState.RUN_ONCE, true);
}
private void run(ScheduledExecutorService taskScheduler, long administrativeYieldMillis, long timeoutMillis, Supplier<ProcessContext> processContextFactory,
SchedulingAgentCallback schedulingAgentCallback, boolean failIfStopping, ScheduledState desiredState, ScheduledState scheduledState) {
private void run(final ScheduledExecutorService taskScheduler, final long administrativeYieldMillis, final long timeoutMillis, final Supplier<ProcessContext> processContextFactory,
final SchedulingAgentCallback schedulingAgentCallback, final boolean failIfStopping, final ScheduledState desiredState,
final ScheduledState scheduledState, final boolean triggerLifecycleMethods) {
final Processor processor = processorRef.get().getProcessor();
final ComponentLog procLog = new SimpleProcessLogger(StandardProcessorNode.this.getIdentifier(), processor, new StandardLoggingContext(StandardProcessorNode.this));
@ -1578,7 +1568,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
}
if (starting) { // will ensure that the Processor represented by this node can only be started once
initiateStart(taskScheduler, administrativeYieldMillis, timeoutMillis, new AtomicLong(0), processContextFactory, schedulingAgentCallback);
initiateStart(taskScheduler, administrativeYieldMillis, timeoutMillis, new AtomicLong(0), processContextFactory, schedulingAgentCallback, triggerLifecycleMethods);
} else {
final String procName = processorRef.get().getProcessor().toString();
procLog.warn("Cannot start {} because it is not currently stopped. Current state is {}", procName, currentState);
@ -1653,6 +1643,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
@Override
public int terminate() {
verifyCanTerminate();
LOG.info("Terminating {}", this);
int count = 0;
for (final Map.Entry<Thread, ActiveTask> entry : activeThreads.entrySet()) {
@ -1669,6 +1660,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
thread.interrupt();
}
LOG.info("Terminated {} threads for {}", count, this);
getLogger().terminate();
completeStopAction();
@ -1694,8 +1686,9 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
}
private void initiateStart(final ScheduledExecutorService taskScheduler, final long administrativeYieldMillis, final long timeoutMilis,
AtomicLong startupAttemptCount, final Supplier<ProcessContext> processContextFactory, final SchedulingAgentCallback schedulingAgentCallback) {
private void initiateStart(final ScheduledExecutorService taskScheduler, final long administrativeYieldMillis, final long timeoutMillis,
final AtomicLong startupAttemptCount, final Supplier<ProcessContext> processContextFactory, final SchedulingAgentCallback schedulingAgentCallback,
final boolean triggerLifecycleMethods) {
final Processor processor = getProcessor();
final ComponentLog procLog = new SimpleProcessLogger(StandardProcessorNode.this.getIdentifier(), processor, new StandardLoggingContext(StandardProcessorNode.this));
@ -1725,17 +1718,17 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
}
// re-initiate the entire process
final Runnable initiateStartTask = () -> initiateStart(taskScheduler, administrativeYieldMillis, timeoutMilis, startupAttemptCount, processContextFactory, schedulingAgentCallback);
final Runnable initiateStartTask = () -> initiateStart(taskScheduler, administrativeYieldMillis, timeoutMillis, startupAttemptCount,
processContextFactory, schedulingAgentCallback, triggerLifecycleMethods);
taskScheduler.schedule(initiateStartTask, 500, TimeUnit.MILLISECONDS);
schedulingAgentCallback.onTaskComplete();
return null;
}
LOG.debug("Invoking @OnScheduled methods of {}", processor);
// Now that the task has been scheduled, set the timeout
completionTimestampRef.set(System.currentTimeMillis() + timeoutMilis);
completionTimestampRef.set(System.currentTimeMillis() + timeoutMillis);
final ProcessContext processContext = processContextFactory.get();
@ -1743,11 +1736,16 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
try {
hasActiveThreads = true;
activateThread();
try {
ReflectionUtils.invokeMethodsWithAnnotation(OnScheduled.class, processor, processContext);
} finally {
deactivateThread();
if (triggerLifecycleMethods) {
LOG.debug("Invoking @OnScheduled methods of {}", processor);
activateThread();
try {
ReflectionUtils.invokeMethodsWithAnnotation(OnScheduled.class, processor, processContext);
} finally {
deactivateThread();
}
} else {
LOG.debug("Will not invoke @OnScheduled methods of {} because triggerLifecycleMethods = false", processor);
}
if (
@ -1760,14 +1758,18 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
LOG.info("Successfully invoked @OnScheduled methods of {} but scheduled state is no longer STARTING so will stop processor now; current state = {}, desired state = {}",
processor, scheduledState.get(), desiredState);
// can only happen if stopProcessor was called before service was transitioned to RUNNING state
activateThread();
try {
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnUnscheduled.class, processor, processContext);
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, processor, processContext);
hasActiveThreads = false;
} finally {
deactivateThread();
if (triggerLifecycleMethods) {
// can only happen if stopProcessor was called before service was transitioned to RUNNING state
activateThread();
try {
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnUnscheduled.class, processor, processContext);
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, processor, processContext);
hasActiveThreads = false;
} finally {
deactivateThread();
}
} else {
LOG.debug("Will not trigger @OnUnscheduled / @OnStopped methods on {} because triggerLifecycleMethods = false", processor);
}
completeStopAction();
@ -1802,7 +1804,9 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
// make sure we only continue retry loop if STOP action wasn't initiated
if (scheduledState.get() != ScheduledState.STOPPING && scheduledState.get() != ScheduledState.RUN_ONCE) {
// re-initiate the entire process
final Runnable initiateStartTask = () -> initiateStart(taskScheduler, administrativeYieldMillis, timeoutMilis, startupAttemptCount, processContextFactory, schedulingAgentCallback);
final Runnable initiateStartTask = () -> initiateStart(taskScheduler, administrativeYieldMillis, timeoutMillis, startupAttemptCount,
processContextFactory, schedulingAgentCallback, triggerLifecycleMethods);
taskScheduler.schedule(initiateStartTask, administrativeYieldMillis, TimeUnit.MILLISECONDS);
} else {
completeStopAction();
@ -1864,7 +1868,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
* STOPPING (e.g., the processor didn't finish @OnScheduled operation when
* stop was called), the attempt will be made to transition processor's
* scheduled state from STARTING to STOPPING which will allow
* {@link #start(ScheduledExecutorService, long, long, Supplier, SchedulingAgentCallback, boolean)}
* {@link #start(ScheduledExecutorService, long, long, Supplier, SchedulingAgentCallback, boolean, boolean)}
* method to initiate processor's shutdown upon exiting @OnScheduled
* operation, otherwise the processor's scheduled state will remain
* unchanged ensuring that multiple calls to this method are idempotent.
@ -1872,7 +1876,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
*/
@Override
public CompletableFuture<Void> stop(final ProcessScheduler processScheduler, final ScheduledExecutorService executor, final ProcessContext processContext,
final SchedulingAgent schedulingAgent, final LifecycleState lifecycleState) {
final SchedulingAgent schedulingAgent, final LifecycleState lifecycleState, final boolean triggerLifecycleMethods) {
final Processor processor = processorRef.get().getProcessor();
LOG.info("Stopping processor: " + this);
@ -1895,11 +1899,17 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
if (lifecycleState.isScheduled()) {
schedulingAgent.unschedule(StandardProcessorNode.this, lifecycleState);
activateThread();
try (final NarCloseable nc = NarCloseable.withComponentNarLoader(getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnUnscheduled.class, processor, processContext);
} finally {
deactivateThread();
if (triggerLifecycleMethods) {
LOG.debug("Triggering @OnUnscheduled methods of {}", this);
activateThread();
try (final NarCloseable nc = NarCloseable.withComponentNarLoader(getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnUnscheduled.class, processor, processContext);
} finally {
deactivateThread();
}
} else {
LOG.debug("Will not trigger @OnUnscheduled methods of {} because triggerLifecycleState = false", this);
}
}
@ -1907,11 +1917,17 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
// performing the lifecycle actions counts as 1 thread.
final boolean allThreadsComplete = lifecycleState.getActiveThreadCount() == 1;
if (allThreadsComplete) {
activateThread();
try (final NarCloseable nc = NarCloseable.withComponentNarLoader(getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, processor, processContext);
} finally {
deactivateThread();
if (triggerLifecycleMethods) {
LOG.debug("Triggering @OnStopped methods of {}", this);
activateThread();
try (final NarCloseable nc = NarCloseable.withComponentNarLoader(getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, processor, processContext);
} finally {
deactivateThread();
}
} else {
LOG.debug("Will not trigger @OnStopped methods of {} because triggerLifecycleState = false", this);
}
lifecycleState.decrementActiveThreadCount();
@ -1924,7 +1940,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
final ScheduledState desired = StandardProcessorNode.this.desiredState;
if (desired == ScheduledState.RUNNING) {
LOG.info("Finished stopping {} but desired state is now RUNNING so will start processor", this);
processScheduler.startProcessor(StandardProcessorNode.this, true);
getProcessGroup().startProcessor(StandardProcessorNode.this, true);
} else if (desired == ScheduledState.DISABLED) {
final boolean updated = scheduledState.compareAndSet(ScheduledState.STOPPED, ScheduledState.DISABLED);
@ -2153,8 +2169,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
continue;
}
serviceNode.removeReference(this, descriptor);
serviceNode.addReference(this, descriptor);
serviceNode.updateReference(this, descriptor);
}
}
}

View File

@ -116,11 +116,9 @@ public abstract class AbstractFlowManager implements FlowManager {
}
public void onProcessorRemoved(final ProcessorNode procNode) {
String identifier = procNode.getIdentifier();
final String identifier = procNode.getIdentifier();
flowFileEventRepository.purgeTransferEvents(identifier);
allProcessors.remove(identifier);
final String version = procNode.getBundleCoordinate().getVersion();
pythonBridge.onProcessorRemoved(identifier, procNode.getComponentType(), procNode.getBundleCoordinate().getVersion());
}
@ -151,6 +149,11 @@ public abstract class AbstractFlowManager implements FlowManager {
return funnel;
}
final ProcessGroup group = getGroup(id);
if (group != null) {
return group.getStatelessGroupNode().orElse(null);
}
final RemoteGroupPort remoteGroupPort = getRootGroup().findRemoteGroupPort(id);
return remoteGroupPort;
}

View File

@ -43,6 +43,7 @@ import org.apache.nifi.controller.TerminationAwareLogger;
import org.apache.nifi.controller.ValidationContextFactory;
import org.apache.nifi.controller.service.ControllerServiceProvider;
import org.apache.nifi.controller.service.StandardConfigurationContext;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.InstanceClassLoader;
@ -71,6 +72,7 @@ import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.Lock;
@ -187,6 +189,11 @@ public class StandardParameterProviderNode extends AbstractComponentNode impleme
return true;
}
@Override
public Optional<ProcessGroup> getParentProcessGroup() {
return Optional.empty();
}
@Override
public ConfigurationContext getConfigurationContext() {
return new StandardConfigurationContext(this, serviceLookup, null, getVariableRegistry());

View File

@ -42,6 +42,7 @@ import org.apache.nifi.controller.scheduling.LifecycleState;
import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.controller.service.ControllerServiceProvider;
import org.apache.nifi.controller.service.StandardConfigurationContext;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.InstanceClassLoader;
@ -63,6 +64,7 @@ import java.net.URL;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
@ -419,4 +421,7 @@ public abstract class AbstractReportingTaskNode extends AbstractComponentNode im
}
}
public Optional<ProcessGroup> getParentProcessGroup() {
return Optional.empty();
}
}

View File

@ -514,11 +514,10 @@ public class StandardProcessSession implements ProcessSession, ProvenanceEventEn
try {
commit(true);
} catch (final Throwable t) {
LOG.error("Failed to asynchronously commit session {} for {}", this, connectableDescription, t);
try {
rollback();
} catch (final Throwable t2) {
t.addSuppressed(t2);
LOG.error("Failed to roll back session {} for {}", this, connectableDescription, t2);
}
@ -1445,6 +1444,7 @@ public class StandardProcessSession implements ProcessSession, ProvenanceEventEn
}
}
@Override
public void migrate(final ProcessSession newOwner) {
final List<FlowFile> allFlowFiles = new ArrayList<>();
@ -2021,7 +2021,7 @@ public class StandardProcessSession implements ProcessSession, ProvenanceEventEn
final FlowFileRecord fFile = new StandardFlowFileRecord.Builder().id(context.getNextFlowFileSequence())
.addAttributes(attrs)
.build();
final StandardRepositoryRecord record = new StandardRepositoryRecord(null);
final StandardRepositoryRecord record = new StandardRepositoryRecord((FlowFileQueue) null);
record.setWorking(fFile, attrs, false);
records.put(fFile.getId(), record);
@ -2063,7 +2063,7 @@ public class StandardProcessSession implements ProcessSession, ProvenanceEventEn
fFileBuilder.addAttributes(newAttributes);
final FlowFileRecord fFile = fFileBuilder.build();
final StandardRepositoryRecord record = new StandardRepositoryRecord(null);
final StandardRepositoryRecord record = new StandardRepositoryRecord((FlowFileQueue) null);
record.setWorking(fFile, newAttributes, false);
records.put(fFile.getId(), record);
createdFlowFiles.add(fFile.getAttribute(CoreAttributes.UUID.key()));
@ -2113,7 +2113,7 @@ public class StandardProcessSession implements ProcessSession, ProvenanceEventEn
.lineageStart(lineageStartDate, lineageStartIndex)
.build();
final StandardRepositoryRecord record = new StandardRepositoryRecord(null);
final StandardRepositoryRecord record = new StandardRepositoryRecord((FlowFileQueue) null);
record.setWorking(fFile, newAttributes, false);
records.put(fFile.getId(), record);
createdFlowFiles.add(fFile.getAttribute(CoreAttributes.UUID.key()));
@ -2145,7 +2145,7 @@ public class StandardProcessSession implements ProcessSession, ProvenanceEventEn
final FlowFileRecord currRec = exampleRepoRecord.getCurrent();
final ContentClaim claim = exampleRepoRecord.getCurrentClaim();
if (offset + size > example.getSize()) {
throw new FlowFileHandlingException("Specified offset of " + offset + " and size " + size + " exceeds size of " + example.toString());
throw new FlowFileHandlingException("Specified offset of " + offset + " and size " + size + " exceeds size of " + example);
}
final StandardFlowFileRecord.Builder builder = new StandardFlowFileRecord.Builder().fromFlowFile(currRec);
@ -2160,7 +2160,7 @@ public class StandardProcessSession implements ProcessSession, ProvenanceEventEn
if (claim != null) {
context.getContentRepository().incrementClaimaintCount(claim);
}
final StandardRepositoryRecord record = new StandardRepositoryRecord(null);
final StandardRepositoryRecord record = new StandardRepositoryRecord((FlowFileQueue) null);
record.setWorking(clone, clone.getAttributes(), false);
records.put(clone.getId(), record);

View File

@ -17,13 +17,13 @@
package org.apache.nifi.controller.repository;
import java.util.Map;
import java.util.WeakHashMap;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.ProcessSessionFactory;
import org.apache.nifi.processor.exception.TerminatedTaskException;
import java.util.Map;
import java.util.WeakHashMap;
public class WeakHashMapProcessSessionFactory implements ActiveProcessSessionFactory {
private final ProcessSessionFactory delegate;
private final Map<ProcessSession, Boolean> sessionMap = new WeakHashMap<>();

View File

@ -0,0 +1,90 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.scheduling;
import java.util.Objects;
import java.util.Optional;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
public class StandardLifecycleStateManager implements LifecycleStateManager {
private final ConcurrentMap<String, LifecycleState> lifecycleStates = new ConcurrentHashMap<>();
@Override
public LifecycleState getOrRegisterLifecycleState(final String componentId, final boolean replaceTerminatedState, final boolean replaceUnscheduledState) {
Objects.requireNonNull(componentId);
LifecycleState lifecycleState;
while (true) {
lifecycleState = this.lifecycleStates.get(componentId);
if (lifecycleState == null) {
lifecycleState = new LifecycleState(componentId);
final LifecycleState existing = this.lifecycleStates.putIfAbsent(componentId, lifecycleState);
if (existing == null) {
break;
} else {
continue;
}
} else if (isReplace(lifecycleState, replaceTerminatedState, replaceUnscheduledState)) {
final LifecycleState newLifecycleState = new LifecycleState(componentId);
final boolean replaced = this.lifecycleStates.replace(componentId, lifecycleState, newLifecycleState);
if (replaced) {
lifecycleState = newLifecycleState;
break;
} else {
continue;
}
} else {
break;
}
}
return lifecycleState;
}
private boolean isReplace(final LifecycleState lifecycleState, final boolean replaceTerminated, final boolean replaceUnscheduled) {
if (replaceTerminated && lifecycleState.isTerminated()) {
return true;
}
if (replaceUnscheduled && !lifecycleState.isScheduled()) {
return true;
}
return false;
}
@Override
public Optional<LifecycleState> getLifecycleState(final String componentId) {
if (componentId == null) {
return Optional.empty();
}
return Optional.ofNullable(lifecycleStates.get(componentId));
}
@Override
public Optional<LifecycleState> removeLifecycleState(final String componentId) {
if (componentId == null) {
return Optional.empty();
}
return Optional.ofNullable(lifecycleStates.remove(componentId));
}
}

View File

@ -77,6 +77,7 @@ import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
@ -95,7 +96,6 @@ public class StandardControllerServiceNode extends AbstractComponentNode impleme
private static final Logger LOG = LoggerFactory.getLogger(StandardControllerServiceNode.class);
public static final String BULLETIN_OBSERVER_ID = "bulletin-observer";
private final AtomicReference<ControllerServiceDetails> controllerServiceHolder = new AtomicReference<>(null);
private final ControllerServiceProvider serviceProvider;
@ -242,6 +242,10 @@ public class StandardControllerServiceNode extends AbstractComponentNode impleme
return processGroup;
}
public Optional<ProcessGroup> getParentProcessGroup() {
return Optional.ofNullable(this.processGroup);
}
@Override
public void setProcessGroup(final ProcessGroup group) {
this.processGroup = group;
@ -264,12 +268,41 @@ public class StandardControllerServiceNode extends AbstractComponentNode impleme
public void addReference(final ComponentNode referencingComponent, final PropertyDescriptor propertyDescriptor) {
writeLock.lock();
try {
referencingComponents.add(new Tuple<>(referencingComponent, propertyDescriptor));
final boolean added = referencingComponents.add(new Tuple<>(referencingComponent, propertyDescriptor));
if (added) {
LOG.debug("{} Added referencing component {} for property {}", this, referencingComponent, propertyDescriptor.getName());
} else {
LOG.debug("{} Will not add referencing component {} for property {} because there is already a reference", this, referencingComponent, propertyDescriptor.getName());
}
} finally {
writeLock.unlock();
}
}
@Override
public void updateReference(final ComponentNode referencingComponent, final PropertyDescriptor propertyDescriptor) {
writeLock.lock();
try {
Tuple<ComponentNode, PropertyDescriptor> updatedTuple = new Tuple<>(referencingComponent, propertyDescriptor);
// Check to see if there is any reference for the given component and property descriptor. If there is, we want to use
// the existing component object instead of the newly provided one. This is done because when a Stateless Process Group is started,
// the instance that is "ethereal" in the stateless engine may call updateReference, and we do not want to change the reference to the
// component to this ethereal instance but instead want to hold onto the existing component.
for (final Tuple<ComponentNode, PropertyDescriptor> tuple : referencingComponents) {
if (Objects.equals(tuple.getKey(), referencingComponent)) {
updatedTuple = new Tuple<>(tuple.getKey(), propertyDescriptor);
LOG.debug("{} updating reference from component {} and property {}", this, referencingComponent, propertyDescriptor);
}
}
referencingComponents.remove(updatedTuple);
referencingComponents.add(updatedTuple);
} finally {
writeLock.unlock();
}
}
@Override
protected ParameterContext getParameterContext() {
@ -783,7 +816,7 @@ public class StandardControllerServiceNode extends AbstractComponentNode impleme
level = LogLevel.WARN;
}
LogRepositoryFactory.getRepository(getIdentifier()).setObservationLevel(BULLETIN_OBSERVER_ID, level);
LogRepositoryFactory.getRepository(getIdentifier()).setObservationLevel(level);
this.bulletinLevel = level;
}

View File

@ -400,13 +400,14 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
}
@Override
public Future<Void> disableControllerServicesAsync(final Collection<ControllerServiceNode> serviceNodes) {
public CompletableFuture<Void> disableControllerServicesAsync(final Collection<ControllerServiceNode> serviceNodes) {
final CompletableFuture<Void> future = new CompletableFuture<>();
processScheduler.submitFrameworkTask(() -> {
try {
disableControllerServices(serviceNodes, future);
future.complete(null);
} catch (final Exception e) {
logger.error("Failed to disable Controller Services {}", serviceNodes, e);
future.completeExceptionally(e);
}
});
@ -462,7 +463,10 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
}
private void disableControllerServiceAndReferencingServices(final ControllerServiceNode serviceNode, final BooleanSupplier cancelSupplier) throws ExecutionException, InterruptedException {
logger.debug("Disabling referencing services for {} before disabling service", serviceNode);
disableReferencingServices(serviceNode);
logger.debug("Disabling service {}", serviceNode);
final CompletableFuture<?> serviceFuture = disableControllerService(serviceNode);
while (true) {
@ -475,6 +479,8 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
}
}
}
logger.debug("{} is now disabled", serviceNode);
}
@Override
@ -701,7 +707,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
for (final ProcessorNode procNode : referencingProcessors) {
if (procNode.getScheduledState() != ScheduledState.DISABLED) {
procNode.verifyCanStart(referencingServiceSet);
procNode.verifyCanStart();
}
}
}

View File

@ -51,6 +51,7 @@ import org.apache.nifi.flow.Bundle;
import org.apache.nifi.flow.ComponentType;
import org.apache.nifi.flow.ConnectableComponent;
import org.apache.nifi.flow.ConnectableComponentType;
import org.apache.nifi.flow.ExecutionEngine;
import org.apache.nifi.flow.ParameterProviderReference;
import org.apache.nifi.flow.VersionedComponent;
import org.apache.nifi.flow.VersionedConnection;
@ -184,7 +185,7 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
final ComparableDataFlow proposedFlow = new StandardComparableDataFlow("Proposed Flow", versionedExternalFlow.getFlowContents());
final PropertyDecryptor decryptor = options.getPropertyDecryptor();
final FlowComparator flowComparator = new StandardFlowComparator(proposedFlow, localFlow, group.getAncestorServiceIds(),
final FlowComparator flowComparator = new StandardFlowComparator(localFlow, proposedFlow, group.getAncestorServiceIds(),
new StaticDifferenceDescriptor(), decryptor::decrypt, options.getComponentComparisonIdLookup(), FlowComparatorVersionedStrategy.DEEP);
final FlowComparison flowComparison = flowComparator.compare();
@ -321,6 +322,22 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
group.setLogFileSuffix(proposed.getLogFileSuffix());
}
final ExecutionEngine proposedExecutionEngine = proposed.getExecutionEngine();
if (proposedExecutionEngine != null) {
group.setExecutionEngine(proposedExecutionEngine);
}
final Integer maxConcurrentTasks = proposed.getMaxConcurrentTasks();
if (maxConcurrentTasks != null) {
group.setMaxConcurrentTasks(maxConcurrentTasks);
}
final String statelessTimeout = proposed.getStatelessFlowTimeout();
if (statelessTimeout != null) {
group.setStatelessFlowTimeout(statelessTimeout);
}
if (proposed.getScheduledState() != null && ScheduledState.RUNNING.name().equals(proposed.getScheduledState().name()) ) {
context.getComponentScheduler().startStatelessGroup(group);
}
final VersionedFlowCoordinates remoteCoordinates = proposed.getVersionedFlowCoordinates();
if (remoteCoordinates == null) {
group.disconnectVersionControl(false);
@ -2412,6 +2429,9 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
port.setName(name);
port.setPosition(new Position(proposed.getPosition().getX(), proposed.getPosition().getY()));
port.setMaxConcurrentTasks(proposed.getConcurrentlySchedulableTaskCount());
if (proposed.getPortFunction() != null) {
port.setPortFunction(proposed.getPortFunction());
}
context.getComponentScheduler().transitionComponentState(port, proposed.getScheduledState());
notifyScheduledStateChange(port, syncOptions, proposed.getScheduledState());
@ -2430,6 +2450,10 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
}
port.setVersionedComponentId(proposed.getIdentifier());
if (proposed.getPortFunction() != null) {
port.setPortFunction(proposed.getPortFunction());
}
destination.addInputPort(port);
updatePort(port, proposed, temporaryName);
connectableAdditionTracker.addComponent(destination.getIdentifier(), proposed.getIdentifier(), port);
@ -2449,6 +2473,9 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
}
port.setVersionedComponentId(proposed.getIdentifier());
if (proposed.getPortFunction() != null) {
port.setPortFunction(proposed.getPortFunction());
}
destination.addOutputPort(port);
updatePort(port, proposed, temporaryName);
connectableAdditionTracker.addComponent(destination.getIdentifier(), proposed.getIdentifier(), port);

View File

@ -20,15 +20,16 @@ package org.apache.nifi.groups;
import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.connectable.Port;
import org.apache.nifi.controller.ProcessorNode;
import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.controller.service.ControllerServiceProvider;
import org.apache.nifi.registry.flow.mapping.VersionedComponentStateLookup;
import org.apache.nifi.controller.ReportingTaskNode;
import org.apache.nifi.controller.service.ControllerServiceProvider;
import org.apache.nifi.flow.ExecutionEngine;
import org.apache.nifi.registry.flow.mapping.VersionedComponentStateLookup;
import org.apache.nifi.remote.RemoteGroupPort;
import java.util.Collection;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class DefaultComponentScheduler extends AbstractComponentScheduler {
private static final Logger logger = LoggerFactory.getLogger(DefaultComponentScheduler.class);
public DefaultComponentScheduler(final ControllerServiceProvider controllerServiceProvider, final VersionedComponentStateLookup stateLookup) {
super(controllerServiceProvider, stateLookup);
@ -36,6 +37,11 @@ public class DefaultComponentScheduler extends AbstractComponentScheduler {
@Override
protected void startNow(final Connectable component) {
if (ExecutionEngine.STATELESS == component.getProcessGroup().resolveExecutionEngine()) {
logger.info("{} should be running but will not start it because its Process Group is configured to run Stateless", component);
return;
}
switch (component.getConnectableType()) {
case PROCESSOR: {
final ProcessorNode processorNode = (ProcessorNode) component;
@ -60,12 +66,11 @@ public class DefaultComponentScheduler extends AbstractComponentScheduler {
}
}
@Override
protected void enableNow(final Collection<ControllerServiceNode> controllerServices) {
getControllerServiceProvider().enableControllerServices(controllerServices);
}
protected void startNow(final ReportingTaskNode reportingTask) {
reportingTask.start();
}
protected void startNow(final ProcessGroup statelessGroup) {
statelessGroup.startProcessing();
}
}

View File

@ -20,13 +20,14 @@ package org.apache.nifi.groups;
import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.controller.ReportingTaskNode;
import org.apache.nifi.controller.ScheduledState;
import org.apache.nifi.controller.Triggerable;
import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.controller.service.ControllerServiceState;
import org.apache.nifi.flow.ExecutionEngine;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
@ -36,18 +37,18 @@ public class RetainExistingStateComponentScheduler implements ComponentScheduler
private static final Logger logger = LoggerFactory.getLogger(RetainExistingStateComponentScheduler.class);
private final ComponentScheduler delegate;
private final Map<String, ScheduledState> connectableStates;
private final Map<String, ScheduledState> componentStates;
private final Map<String, ControllerServiceState> controllerServiceStates;
public RetainExistingStateComponentScheduler(final ProcessGroup processGroup, final ComponentScheduler delegate) {
this.delegate = delegate;
this.connectableStates = mapConnectableStates(processGroup);
this.componentStates = mapComponentStates(processGroup);
this.controllerServiceStates = mapControllerServiceStates(processGroup);
}
@Override
public void startComponent(final Connectable component) {
final ScheduledState existingState = connectableStates.get(component.getIdentifier());
final ScheduledState existingState = componentStates.get(component.getIdentifier());
if (existingState == null) {
logger.debug("Will not start {} because it was not previously known in this Process Group", component);
return;
@ -62,6 +63,28 @@ public class RetainExistingStateComponentScheduler implements ComponentScheduler
delegate.startComponent(component);
}
@Override
public void startStatelessGroup(final ProcessGroup group) {
final ScheduledState existingState = componentStates.get(group.getIdentifier());
if (existingState == null) {
logger.debug("Will not start {} because it was not previously known in this Process Group", group);
return;
}
if (existingState != ScheduledState.RUNNING && existingState != ScheduledState.STARTING) {
logger.debug("Will not start {} because its previous state was {}", group, existingState);
return;
}
logger.debug("Starting {}", group);
delegate.startStatelessGroup(group);
}
@Override
public void stopStatelessGroup(final ProcessGroup group) {
delegate.stopStatelessGroup(group);
}
@Override
public void stopComponent(final Connectable component) {
delegate.stopComponent(component);
@ -124,14 +147,24 @@ public class RetainExistingStateComponentScheduler implements ComponentScheduler
return serviceStates;
}
private Map<String, ScheduledState> mapConnectableStates(final ProcessGroup group) {
private Map<String, ScheduledState> mapComponentStates(final ProcessGroup group) {
final Set<Connectable> connectables = new HashSet<>();
findAllConnectables(group, connectables);
final Map<String, ScheduledState> connectableStates = connectables.stream()
.collect(Collectors.toMap(Connectable::getIdentifier, Triggerable::getScheduledState));
final Map<String, ScheduledState> componentStates = new HashMap<>();
for (final Connectable connectable : connectables) {
componentStates.put(connectable.getIdentifier(), connectable.getScheduledState());
}
return connectableStates;
final Set<ProcessGroup> statelessGroups = new HashSet<>();
findAllStatelessGroups(group, statelessGroups);
for (final ProcessGroup statelessGroup : statelessGroups) {
final StatelessGroupScheduledState state = statelessGroup.getStatelessScheduledState();
final ScheduledState scheduledState = state == StatelessGroupScheduledState.RUNNING ? ScheduledState.RUNNING : ScheduledState.STOPPED;
componentStates.put(statelessGroup.getIdentifier(), scheduledState);
}
return componentStates;
}
private void findAllConnectables(final ProcessGroup group, final Set<Connectable> connectables) {
@ -148,4 +181,15 @@ public class RetainExistingStateComponentScheduler implements ComponentScheduler
findAllConnectables(childGroup, connectables);
}
}
private void findAllStatelessGroups(final ProcessGroup start, final Set<ProcessGroup> statelessGroups) {
if (start.resolveExecutionEngine() == ExecutionEngine.STATELESS) {
statelessGroups.add(start);
return; // No need to go further, as the top-level stateless group is all we need.
}
for (final ProcessGroup childGroup : start.getProcessGroups()) {
findAllStatelessGroups(childGroup, statelessGroups);
}
}
}

View File

@ -64,6 +64,7 @@ import org.apache.nifi.controller.service.ControllerServiceReference;
import org.apache.nifi.controller.service.ControllerServiceState;
import org.apache.nifi.controller.service.StandardConfigurationContext;
import org.apache.nifi.encrypt.PropertyEncryptor;
import org.apache.nifi.flow.ExecutionEngine;
import org.apache.nifi.flow.VersionedComponent;
import org.apache.nifi.flow.VersionedExternalFlow;
import org.apache.nifi.flow.VersionedProcessGroup;
@ -197,6 +198,10 @@ public final class StandardProcessGroup implements ProcessGroup {
private final VersionControlFields versionControlFields = new VersionControlFields();
private volatile ParameterContext parameterContext;
private final NodeTypeProvider nodeTypeProvider;
private final StatelessGroupNode statelessGroupNode;
private volatile ExecutionEngine executionEngine = ExecutionEngine.INHERITED;
private volatile int maxConcurrentTasks = 1;
private volatile String statelessFlowTimeout = "1 min";
private FlowFileConcurrency flowFileConcurrency = FlowFileConcurrency.UNBOUNDED;
private volatile FlowFileGate flowFileGate = new UnboundedFlowFileGate();
@ -222,7 +227,7 @@ public final class StandardProcessGroup implements ProcessGroup {
final PropertyEncryptor encryptor, final ExtensionManager extensionManager,
final StateManagerProvider stateManagerProvider, final FlowManager flowManager,
final ReloadComponent reloadComponent, final MutableVariableRegistry variableRegistry, final NodeTypeProvider nodeTypeProvider,
final NiFiProperties nifiProperties) {
final NiFiProperties nifiProperties, final StatelessGroupNodeFactory statelessGroupNodeFactory) {
this.id = id;
this.controllerServiceProvider = serviceProvider;
@ -274,8 +279,9 @@ public final class StandardProcessGroup implements ProcessGroup {
}
nifiPropertiesBackpressureSize = size;
}
}
statelessGroupNode = statelessGroupNodeFactory.createStatelessGroupNode(this);
}
@Override
public ProcessGroup getParent() {
@ -510,12 +516,47 @@ public final class StandardProcessGroup implements ProcessGroup {
return parent.get() == null;
}
@Override
public void startProcessing() {
final ExecutionEngine resolvedExecutionEngine = resolveExecutionEngine();
if (resolvedExecutionEngine == ExecutionEngine.STATELESS) {
writeLock.lock();
try {
final ProcessGroup parent = getParent();
if (parent != null) {
final ExecutionEngine parentExecutionEngine = parent.resolveExecutionEngine();
if (parentExecutionEngine == ExecutionEngine.STATELESS) {
LOG.warn("Cannot start Process Group {} because its parent is configured to run using the Stateless Engine. Only the top-most Process Group that is " +
"configured to use the Stateless Engine may be directly started", this);
return;
}
}
if (getStatelessScheduledState() == StatelessGroupScheduledState.RUNNING) {
LOG.info("Triggered to start {} but it is already running", this);
return;
}
scheduler.startStatelessGroup(statelessGroupNode);
LOG.info("Started {} to run as a Stateless Process Group", this);
return;
} finally {
writeLock.unlock();
}
}
startComponents();
onComponentModified();
}
@Override
public void startComponents() {
readLock.lock();
try {
enableAllControllerServices();
findAllProcessors().stream().filter(START_PROCESSORS_FILTER).forEach(node -> {
controllerServiceProvider.enableControllerServices(controllerServices.values());
getProcessors().stream().filter(START_PROCESSORS_FILTER).forEach(node -> {
try {
node.getProcessGroup().startProcessor(node, true);
} catch (final Throwable t) {
@ -523,35 +564,114 @@ public final class StandardProcessGroup implements ProcessGroup {
}
});
findAllInputPorts().stream().filter(START_PORTS_FILTER).forEach(port -> port.getProcessGroup().startInputPort(port));
getInputPorts().stream().filter(START_PORTS_FILTER).forEach(port -> port.getProcessGroup().startInputPort(port));
getOutputPorts().stream().filter(START_PORTS_FILTER).forEach(port -> port.getProcessGroup().startOutputPort(port));
findAllOutputPorts().stream().filter(START_PORTS_FILTER).forEach(port -> port.getProcessGroup().startOutputPort(port));
getProcessGroups().forEach(ProcessGroup::startProcessing);
} finally {
readLock.unlock();
}
onComponentModified();
}
@Override
public void stopProcessing() {
public CompletableFuture<Void> stopProcessing() {
if (resolveExecutionEngine() == ExecutionEngine.STATELESS) {
writeLock.lock();
try {
final ProcessGroup parentStatelessGroup = getStatelessGroup(getParent());
if (parentStatelessGroup != null) {
// This is not the top-level stateless group. Nothing to do.
return CompletableFuture.completedFuture(null);
}
LOG.info("Stopping {} from running", this);
final CompletableFuture<Void> future = scheduler.stopStatelessGroup(statelessGroupNode);
return future;
} finally {
writeLock.unlock();
}
}
final CompletableFuture<Void> stopComponentsFuture = stopComponents();
onComponentModified();
return stopComponentsFuture;
}
@Override
public CompletableFuture<Void> stopComponents() {
readLock.lock();
try {
findAllProcessors().stream().filter(STOP_PROCESSORS_FILTER).forEach(node -> {
final List<CompletableFuture<Void>> futures = new ArrayList<>();
getProcessors().stream().filter(STOP_PROCESSORS_FILTER).forEach(node -> {
try {
node.getProcessGroup().stopProcessor(node);
futures.add(node.getProcessGroup().stopProcessor(node));
} catch (final Throwable t) {
LOG.error("Unable to stop processor {}", node.getIdentifier(), t);
}
});
findAllInputPorts().stream().filter(STOP_PORTS_FILTER).forEach(port -> port.getProcessGroup().stopInputPort(port));
findAllOutputPorts().stream().filter(STOP_PORTS_FILTER).forEach(port -> port.getProcessGroup().stopOutputPort(port));
getInputPorts().stream().filter(STOP_PORTS_FILTER).forEach(port -> port.getProcessGroup().stopInputPort(port));
getOutputPorts().stream().filter(STOP_PORTS_FILTER).forEach(port -> port.getProcessGroup().stopOutputPort(port));
for (final ProcessGroup childGroup : getProcessGroups()) {
final CompletableFuture<Void> future = childGroup.stopProcessing();
futures.add(future);
}
return CompletableFuture.allOf(futures.toArray(new CompletableFuture[0]));
} finally {
readLock.unlock();
}
}
onComponentModified();
@Override
public StatelessGroupScheduledState getStatelessScheduledState() {
if (statelessGroupNode == null) {
return StatelessGroupScheduledState.STOPPED;
}
final ScheduledState currentState = statelessGroupNode.getCurrentState();
switch (currentState) {
case RUNNING:
case RUN_ONCE:
case STARTING:
case STOPPING:
return StatelessGroupScheduledState.RUNNING;
default:
return StatelessGroupScheduledState.STOPPED;
}
}
@Override
public StatelessGroupScheduledState getDesiredStatelessScheduledState() {
if (statelessGroupNode == null) {
return StatelessGroupScheduledState.STOPPED;
}
final ScheduledState currentState = statelessGroupNode.getDesiredState();
switch (currentState) {
case RUNNING:
case STARTING:
return StatelessGroupScheduledState.RUNNING;
default:
return StatelessGroupScheduledState.STOPPED;
}
}
@Override
public boolean isStatelessActive() {
if (statelessGroupNode == null) {
return false;
}
if (getStatelessScheduledState() == StatelessGroupScheduledState.RUNNING) {
return true;
}
return this.scheduler.getActiveThreadCount(statelessGroupNode) > 0;
}
private StateManager getStateManager(final String componentId) {
@ -832,6 +952,7 @@ public final class StandardProcessGroup implements ProcessGroup {
onComponentModified();
flowManager.onProcessGroupRemoved(group);
LogRepositoryFactory.removeRepository(group.getIdentifier());
LOG.info("{} removed from flow", group);
} finally {
writeLock.unlock();
@ -1614,6 +1735,8 @@ public final class StandardProcessGroup implements ProcessGroup {
throw new IllegalStateException("Processor is not a member of this Process Group");
}
verifyCanStart(processor);
final ScheduledState state = processor.getScheduledState();
if (state == ScheduledState.DISABLED) {
throw new IllegalStateException("Processor is disabled");
@ -1662,6 +1785,8 @@ public final class StandardProcessGroup implements ProcessGroup {
throw new IllegalStateException("Port " + port.getIdentifier() + " is not a member of this Process Group");
}
verifyCanStart(port);
final ScheduledState state = port.getScheduledState();
if (state == ScheduledState.DISABLED) {
throw new IllegalStateException("InputPort " + port.getIdentifier() + " is disabled");
@ -1683,6 +1808,8 @@ public final class StandardProcessGroup implements ProcessGroup {
throw new IllegalStateException("Port is not a member of this Process Group");
}
verifyCanStart(port);
final ScheduledState state = port.getScheduledState();
if (state == ScheduledState.DISABLED) {
throw new IllegalStateException("OutputPort is disabled");
@ -1715,7 +1842,7 @@ public final class StandardProcessGroup implements ProcessGroup {
}
@Override
public Future<Void> stopProcessor(final ProcessorNode processor) {
public CompletableFuture<Void> stopProcessor(final ProcessorNode processor) {
readLock.lock();
try {
if (!processors.containsKey(processor.getIdentifier())) {
@ -1878,17 +2005,6 @@ public final class StandardProcessGroup implements ProcessGroup {
}
}
@Override
public void enableAllControllerServices() {
// Enable all valid controller services in this process group
controllerServiceProvider.enableControllerServices(controllerServices.values());
// Enable all controller services for child process groups
for (ProcessGroup pg : processGroups.values()) {
pg.enableAllControllerServices();
}
}
@Override
public void disableInputPort(final Port port) {
readLock.lock();
@ -3027,18 +3143,9 @@ public final class StandardProcessGroup implements ProcessGroup {
}
@Override
public void verifyCanStart(Connectable connectable) {
readLock.lock();
try {
if (connectable.getScheduledState() == ScheduledState.STOPPED) {
if (scheduler.getActiveThreadCount(connectable) > 0) {
throw new IllegalStateException("Cannot start component with id" + connectable.getIdentifier() + " because it is currently stopping");
}
connectable.verifyCanStart();
}
} finally {
readLock.unlock();
public void verifyCanStart(final Connectable connectable) {
if (connectable.getScheduledState() == ScheduledState.STOPPED) {
connectable.verifyCanStart();
}
}
@ -3049,11 +3156,24 @@ public final class StandardProcessGroup implements ProcessGroup {
for (final Connectable connectable : findAllConnectables(this, false)) {
verifyCanStart(connectable);
}
final Set<ControllerServiceNode> services = findAllControllerServices();
for (final ControllerServiceNode serviceNode : services) {
serviceNode.verifyCanEnable(services);
}
} finally {
readLock.unlock();
}
}
@Override
public void verifyCanScheduleComponentsIndividually() {
if (resolveExecutionEngine() == ExecutionEngine.STATELESS) {
throw new IllegalStateException("Cannot schedule components individually because the Process Group is configured to run in Stateless mode.");
}
}
@Override
public void verifyCanDelete(final Snippet snippet) throws IllegalStateException {
readLock.lock();
@ -3151,6 +3271,9 @@ public final class StandardProcessGroup implements ProcessGroup {
throw new IllegalStateException("One or more components within the snippet is connected to a component outside of the snippet. Only a disconnected snippet may be moved.");
}
final ExecutionEngine newGroupExecutionEngine = newProcessGroup.resolveExecutionEngine();
final ExecutionEngine executionEngine = resolveExecutionEngine();
for (final String id : snippet.getInputPorts().keySet()) {
final Port port = getInputPort(id);
final String portName = port.getName();
@ -3158,6 +3281,11 @@ public final class StandardProcessGroup implements ProcessGroup {
if (newProcessGroup.getInputPortByName(portName) != null) {
throw new IllegalStateException("Cannot perform Move Operation because of a naming conflict with another port in the destination Process Group");
}
if (newGroupExecutionEngine != executionEngine && port.isRunning()) {
throw new IllegalStateException("Cannot perform Move Operation because Input Port with ID " + port.getIdentifier() + " is running, and the destination Process Group has a " +
"different Execution Engine than the current Process Group. The Port must be stopped before it can be moved to a Process Group with a different Execution Engine.");
}
}
for (final String id : snippet.getOutputPorts().keySet()) {
@ -3167,6 +3295,49 @@ public final class StandardProcessGroup implements ProcessGroup {
if (newProcessGroup.getOutputPortByName(portName) != null) {
throw new IllegalStateException("Cannot perform Move Operation because of a naming conflict with another port in the destination Process Group");
}
if (newGroupExecutionEngine != executionEngine && port.isRunning()) {
throw new IllegalStateException("Cannot perform Move Operation because Output Port with ID " + port.getIdentifier() + " is running, and the destination Process Group has a " +
"different Execution Engine than the current Process Group. The Port must be stopped before it can be moved to a Process Group with a different Execution Engine.");
}
}
// Check Execution Engine compatibility
for (final String id : snippet.getProcessGroups().keySet()) {
final ProcessGroup childGroup = getProcessGroup(id);
final ExecutionEngine childEngine = childGroup.resolveExecutionEngine();
if (childEngine == ExecutionEngine.STANDARD && newGroupExecutionEngine != ExecutionEngine.STANDARD) {
throw new IllegalStateException("Cannot move a Process Group that is configured to run with the Traditional Execution Engine " +
" to a Process Group that is configured to run with the Stateless Execution Engine.");
}
if (childEngine == ExecutionEngine.STATELESS && newGroupExecutionEngine == ExecutionEngine.STANDARD
&& childGroup.getStatelessScheduledState() != StatelessGroupScheduledState.STOPPED) {
throw new IllegalStateException("Cannot move a Process Group that is configured to run with the " + childEngine +
" Execution Engine to a Process Group that is configured to run with the " + newGroupExecutionEngine +
" unless all components are stopped");
}
}
if (newGroupExecutionEngine != executionEngine) {
for (final String id : snippet.getProcessors().keySet()) {
final ProcessorNode procNode = getProcessor(id);
if (procNode.isRunning()) {
throw new IllegalStateException("Cannot perform Move Operation because Processor with ID " + procNode.getIdentifier() +
" is running, and the destination Process Group has a different Execution Engine than the current Process Group." +
" The Processor must be stopped before it can be moved to a Process Group with a different Execution Engine.");
}
}
for (final String id : snippet.getRemoteProcessGroups().keySet()) {
final RemoteProcessGroup rpg = getRemoteProcessGroup(id);
if (rpg.isTransmitting()) {
throw new IllegalStateException("Cannot perform Move Operation because Remote Process Group with ID " + rpg.getIdentifier() +
" is running, and the destination Process Group has a different Execution Engine than the current Process Group." +
" The Remote Process Group must be stopped before it can be moved to a Process Group with a different Execution Engine.");
}
}
}
final ParameterContext currentParameterContext = getParameterContext();
@ -3738,6 +3909,9 @@ public final class StandardProcessGroup implements ProcessGroup {
copy.setVariables(processGroup.getVariables());
copy.setLabels(processGroup.getLabels());
copy.setParameterContextName(processGroup.getParameterContextName());
copy.setExecutionEngine(processGroup.getExecutionEngine());
copy.setMaxConcurrentTasks(processGroup.getMaxConcurrentTasks());
copy.setStatelessFlowTimeout(processGroup.getStatelessFlowTimeout());
final Set<VersionedProcessGroup> copyChildren = new HashSet<>();
@ -3759,6 +3933,9 @@ public final class StandardProcessGroup implements ProcessGroup {
childCopy.setDefaultBackPressureObjectThreshold(childGroup.getDefaultBackPressureObjectThreshold());
childCopy.setDefaultBackPressureDataSizeThreshold(childGroup.getDefaultBackPressureDataSizeThreshold());
childCopy.setParameterContextName(childGroup.getParameterContextName());
childCopy.setExecutionEngine(childGroup.getExecutionEngine());
childCopy.setMaxConcurrentTasks(childGroup.getMaxConcurrentTasks());
childCopy.setStatelessFlowTimeout(childGroup.getStatelessFlowTimeout());
copyChildren.add(childCopy);
}
@ -4439,6 +4616,128 @@ public final class StandardProcessGroup implements ProcessGroup {
}
}
public ExecutionEngine getExecutionEngine() {
return executionEngine;
}
@Override
public void setExecutionEngine(final ExecutionEngine executionEngine) {
writeLock.lock();
try {
verifyCanSetExecutionEngine(executionEngine);
this.executionEngine = executionEngine;
} finally {
writeLock.unlock();
}
}
@Override
public Optional<StatelessGroupNode> getStatelessGroupNode() {
return Optional.ofNullable(statelessGroupNode);
}
@Override
public ExecutionEngine resolveExecutionEngine() {
final ExecutionEngine engine = getExecutionEngine();
if (engine == ExecutionEngine.INHERITED) {
final ProcessGroup parent = getParent();
return parent == null ? ExecutionEngine.STANDARD : parent.resolveExecutionEngine();
}
return engine;
}
private ProcessGroup getStatelessGroup(final ProcessGroup start) {
if (start == null) {
return null;
}
final ExecutionEngine engine = start.getExecutionEngine();
if (engine == ExecutionEngine.STATELESS) {
return start;
}
return getStatelessGroup(start.getParent());
}
@Override
public void verifyCanSetExecutionEngine(final ExecutionEngine executionEngine) {
final ExecutionEngine resolvedProposedEngine;
if (Objects.requireNonNull(executionEngine) == ExecutionEngine.INHERITED) {
final ProcessGroup parent = getParent();
resolvedProposedEngine = (parent == null) ? ExecutionEngine.STANDARD : parent.resolveExecutionEngine();
} else {
resolvedProposedEngine = executionEngine;
}
// If unchanged, nothing more to check
if (resolvedProposedEngine == resolveExecutionEngine()) {
LOG.debug("Allowing the setting of Execution Engine to {} because it resolves to the same engine that is currently selected for {}", executionEngine, this);
return;
}
if (executionEngine == ExecutionEngine.STANDARD) {
final ProcessGroup statelessGroup = getStatelessGroup(getParent());
if (statelessGroup != null) {
throw new IllegalStateException("A Process Group using the Standard Engine may not be the child of a Process Group using the Stateless Engine. Cannot set Execution Engine of " + this +
" to Standard because it is a child of " + statelessGroup);
}
}
for (final ProcessorNode processor : getProcessors()) {
if (processor.isRunning()) {
throw new IllegalStateException("Cannot change Execution Engine for " + this + " while components are running. " + processor + " is currently running.");
}
}
for (final Port port : getInputPorts()) {
if (port.isRunning()) {
throw new IllegalStateException("Cannot change Execution Engine for " + this + " while components are running. Input Port " + port + " is currently running.");
}
}
for (final Port port : getOutputPorts()) {
if (port.isRunning()) {
throw new IllegalStateException("Cannot change Execution Engine for " + this + " while components are running. Output Port " + port + " is currently running.");
}
}
for (final RemoteProcessGroup rpg : getRemoteProcessGroups()) {
if (rpg.isTransmitting()) {
throw new IllegalStateException("Cannot change Execution Engine for " + this + " while components are running. " + rpg + " is currently running.");
}
}
for (final ControllerServiceNode service : getControllerServices(false)) {
if (service.isActive()) {
throw new IllegalStateException("Cannot change Execution Engine for " + this + " while Controller Services are active. " + service + " is currently active.");
}
}
for (final Connection connection : getConnections()) {
final boolean queueEmpty = connection.getFlowFileQueue().isEmpty();
if (!queueEmpty) {
throw new IllegalStateException("Cannot change Execution Engine for " + this + " while data is queued. " + connection + " has data queued.");
}
}
for (final ProcessGroup child : getProcessGroups()) {
if (child.getExecutionEngine() == ExecutionEngine.INHERITED) {
child.verifyCanSetExecutionEngine(executionEngine);
}
}
}
@Override
public void setMaxConcurrentTasks(final int maxConcurrentTasks) {
this.maxConcurrentTasks = maxConcurrentTasks;
if (statelessGroupNode != null) {
statelessGroupNode.setMaxConcurrentTasks(maxConcurrentTasks);
}
}
@Override
public int getMaxConcurrentTasks() {
return maxConcurrentTasks;
}
@Override
public String getDefaultBackPressureDataSizeThreshold() {
// Use value in this object if it has been set. Otherwise, inherit from parent group; if at root group, obtain from nifi properties.
@ -4452,4 +4751,23 @@ public final class StandardProcessGroup implements ProcessGroup {
return defaultBackPressureDataSizeThreshold.get();
}
@Override
public String getStatelessFlowTimeout() {
return statelessFlowTimeout;
}
@Override
public void setStatelessFlowTimeout(final String statelessFlowTimeout) {
if (statelessFlowTimeout == null) {
return;
}
try {
FormatUtils.getPreciseTimeDuration(Objects.requireNonNull(statelessFlowTimeout), TimeUnit.MILLISECONDS); // Verify that the value is valid
this.statelessFlowTimeout = statelessFlowTimeout;
} catch (final Exception e) {
LOG.warn("Attempted to set Stateless Flow Timeout for {} to invalid value: {}; ignoring this value", this, statelessFlowTimeout);
}
}
}

View File

@ -0,0 +1,27 @@
/*
* 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.groups;
public interface StatelessGroupNodeFactory {
/**
* Creates a Stateless Group Node for the given ProcessGroup
* @param group the ProcessGroup
* @return a Stateless Group Node that can be used to run the flow using the Stateless Engine
*/
StatelessGroupNode createStatelessGroupNode(ProcessGroup group);
}

View File

@ -0,0 +1,49 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.logging;
import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.events.BulletinFactory;
import org.apache.nifi.reporting.BulletinRepository;
import org.apache.nifi.reporting.Severity;
public class ConnectableLogObserver implements LogObserver {
private static final String CATEGORY = "Log Message";
private final BulletinRepository bulletinRepository;
private final Connectable connectable;
public ConnectableLogObserver(final BulletinRepository bulletinRepository, final Connectable connectable) {
this.bulletinRepository = bulletinRepository;
this.connectable = connectable;
}
@Override
public void onLogMessage(final LogMessage message) {
// Map LogLevel.WARN to Severity.WARNING so that we are consistent with the Severity enumeration. Else, just use whatever
// the LogLevel is (INFO and ERROR map directly and all others we will just accept as they are).
final String bulletinLevel = (message.getLogLevel() == LogLevel.WARN) ? Severity.WARNING.name() : message.getLogLevel().toString();
bulletinRepository.addBulletin(BulletinFactory.createBulletin(connectable, CATEGORY, bulletinLevel, message.getMessage(), message.getFlowFileUuid()));
}
@Override
public String getComponentDescription() {
return connectable.toString();
}
}

View File

@ -23,11 +23,7 @@ import org.apache.nifi.logging.LogMessage;
import org.apache.nifi.logging.LogObserver;
import org.apache.nifi.logging.LogRepository;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
public class NopLogRepository implements LogRepository {
private final ConcurrentMap<String, LogLevel> observationLevels = new ConcurrentHashMap<>();
private volatile ComponentLog logger;
@Override
@ -43,28 +39,16 @@ public class NopLogRepository implements LogRepository {
}
@Override
public void addObserver(final String observerIdentifier, final LogLevel level, final LogObserver observer) {
public void addObserver(final LogLevel level, final LogObserver observer) {
}
@Override
public void setObservationLevel(final String observerIdentifier, final LogLevel level) {
observationLevels.put(observerIdentifier, level);
public void setObservationLevel(final LogLevel level) {
}
@Override
public LogLevel getObservationLevel(final String observerIdentifier) {
return observationLevels.get(observerIdentifier);
}
@Override
public LogObserver removeObserver(final String observerIdentifier) {
observationLevels.remove(observerIdentifier);
return null;
}
@Override
public void removeAllObservers() {
observationLevels.clear();
}
@Override

View File

@ -29,21 +29,20 @@ import org.slf4j.helpers.MessageFormatter;
import java.util.ArrayList;
import java.util.Collection;
import java.util.EnumMap;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.concurrent.locks.ReentrantLock;
public class StandardLogRepository implements LogRepository {
private final Map<LogLevel, Collection<LogObserver>> observers = new EnumMap<>(LogLevel.class);
private final Map<String, LogObserver> observerLookup = new HashMap<>();
private final Map<LogLevel, Collection<LogObserver>> observersPerLogLevel = new ConcurrentHashMap<>();
private final Set<LogObserver> observers = new HashSet<>();
private final ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock();
private final Lock readLock = rwLock.readLock();
private final Lock writeLock = rwLock.writeLock();
private final Lock lock = new ReentrantLock();
private final Logger logger = LoggerFactory.getLogger(StandardLogRepository.class);
@ -53,7 +52,7 @@ public class StandardLogRepository implements LogRepository {
public void addLogMessage(LogMessage logMessage) {
LogLevel logLevel = logMessage.getLogLevel();
final Collection<LogObserver> logObservers = observers.get(logLevel);
final Collection<LogObserver> logObservers = observersPerLogLevel.get(logLevel);
if (logObservers != null) {
for (LogObserver observer : logObservers) {
try {
@ -123,89 +122,48 @@ public class StandardLogRepository implements LogRepository {
}
@Override
public void setObservationLevel(String observerIdentifier, LogLevel level) {
writeLock.lock();
public void setObservationLevel(LogLevel level) {
lock.lock();
try {
final LogObserver observer = removeObserver(observerIdentifier);
final Set<LogObserver> observersCopy = new HashSet<>(observers);
observers.clear();
if (observer != null) {
addObserver(observerIdentifier, level, observer);
for (final LogObserver observer : observersCopy) {
addObserver(level, observer);
}
} finally {
writeLock.unlock();
lock.unlock();
}
}
@Override
public LogLevel getObservationLevel(String observerIdentifier) {
readLock.lock();
try {
// ensure observer exists
if (!observerLookup.containsKey(observerIdentifier)) {
throw new IllegalStateException("The specified observer identifier does not exist.");
}
final LogObserver observer = observerLookup.get(observerIdentifier);
for (final LogLevel logLevel : LogLevel.values()) {
final Collection<LogObserver> levelObservers = observers.get(logLevel);
if (levelObservers != null && levelObservers.contains(observer)) {
return logLevel;
}
}
// at this point, the LogLevel must be NONE since we don't register observers for NONE
return LogLevel.NONE;
} finally {
readLock.unlock();
}
}
@Override
public void addObserver(final String observerIdentifier, final LogLevel minimumLevel, final LogObserver observer) {
writeLock.lock();
public void addObserver(final LogLevel minimumLevel, final LogObserver observer) {
lock.lock();
try {
// ensure observer does not exists
if (observerLookup.containsKey(observerIdentifier)) {
throw new IllegalStateException("Cannot add Log Observer for " + observer.getComponentDescription() +
" because the specified observer identifier (" + observerIdentifier + ") already exists.");
}
final LogLevel[] allLevels = LogLevel.values();
for (int i = minimumLevel.ordinal(); i < allLevels.length; i++) {
// no need to register an observer for NONE since that level will never be logged to by a component
if (i != LogLevel.NONE.ordinal()) {
Collection<LogObserver> collection = observers.computeIfAbsent(allLevels[i], k -> new ArrayList<>());
Collection<LogObserver> collection = observersPerLogLevel.computeIfAbsent(allLevels[i], k -> new ArrayList<>());
collection.add(observer);
}
}
observerLookup.put(observerIdentifier, observer);
observers.add(observer);
} finally {
writeLock.unlock();
lock.unlock();
}
}
@Override
public LogObserver removeObserver(final String observerIdentifier) {
writeLock.lock();
try {
final LogObserver observer = observerLookup.get(observerIdentifier);
for (final Collection<LogObserver> collection : observers.values()) {
collection.remove(observer);
}
return observerLookup.remove(observerIdentifier);
} finally {
writeLock.unlock();
}
}
@Override
public void removeAllObservers() {
writeLock.lock();
lock.lock();
try {
observersPerLogLevel.clear();
observers.clear();
observerLookup.clear();
} finally {
writeLock.unlock();
lock.unlock();
}
}
@ -220,7 +178,7 @@ public class StandardLogRepository implements LogRepository {
}
private boolean hasObserver(final LogLevel logLevel) {
final Collection<LogObserver> logLevelObservers = observers.get(logLevel);
final Collection<LogObserver> logLevelObservers = observersPerLogLevel.get(logLevel);
return (logLevelObservers != null && !logLevelObservers.isEmpty());
}

View File

@ -39,6 +39,7 @@ import org.apache.nifi.flow.ParameterProviderReference;
import org.apache.nifi.flow.VersionedFlowCoordinates;
import org.apache.nifi.flow.VersionedParameterContext;
import org.apache.nifi.flow.VersionedProcessGroup;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.NarCloseable;
import org.apache.nifi.parameter.ParameterContext;
@ -58,6 +59,7 @@ import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;
@ -164,6 +166,11 @@ public final class StandardFlowRegistryClientNode extends AbstractComponentNode
return getValidationStatus() != ValidationStatus.VALID;
}
@Override
public Optional<ProcessGroup> getParentProcessGroup() {
return Optional.empty();
}
@Override
public ParameterLookup getParameterLookup() {
return ParameterLookup.EMPTY;

View File

@ -17,7 +17,6 @@
package org.apache.nifi.registry.flow.mapping;
import org.apache.commons.lang3.ClassUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.bundle.BundleCoordinate;
@ -46,6 +45,7 @@ import org.apache.nifi.flow.ComponentType;
import org.apache.nifi.flow.ConnectableComponent;
import org.apache.nifi.flow.ConnectableComponentType;
import org.apache.nifi.flow.ControllerServiceAPI;
import org.apache.nifi.flow.ExecutionEngine;
import org.apache.nifi.flow.ExternalControllerServiceReference;
import org.apache.nifi.flow.ParameterProviderReference;
import org.apache.nifi.flow.PortType;
@ -260,6 +260,11 @@ public class NiFiRegistryFlowMapper {
versionedGroup.setDefaultBackPressureDataSizeThreshold(group.getDefaultBackPressureDataSizeThreshold());
versionedGroup.setLogFileSuffix(group.getLogFileSuffix());
versionedGroup.setExecutionEngine(ExecutionEngine.valueOf(group.getExecutionEngine().name()));
versionedGroup.setScheduledState(flowMappingOptions.getStateLookup().getState(group));
versionedGroup.setMaxConcurrentTasks(group.getMaxConcurrentTasks());
versionedGroup.setStatelessFlowTimeout(group.getStatelessFlowTimeout());
final ParameterContext parameterContext = group.getParameterContext();
versionedGroup.setParameterContextName(parameterContext == null ? null : parameterContext.getName());
@ -743,6 +748,7 @@ public class NiFiRegistryFlowMapper {
versionedPort.setScheduledState(mapScheduledState(port.getScheduledState()));
versionedPort.setAllowRemoteAccess(port instanceof PublicPort);
versionedPort.setScheduledState(flowMappingOptions.getStateLookup().getState(port));
versionedPort.setPortFunction(port.getPortFunction());
return versionedPort;
}

View File

@ -47,6 +47,7 @@ import org.apache.nifi.controller.status.TransmissionStatus;
import org.apache.nifi.controller.status.analytics.ConnectionStatusPredictions;
import org.apache.nifi.controller.status.analytics.StatusAnalytics;
import org.apache.nifi.controller.status.analytics.StatusAnalyticsEngine;
import org.apache.nifi.flow.ExecutionEngine;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.groups.RemoteProcessGroup;
import org.apache.nifi.processor.Relationship;
@ -475,7 +476,16 @@ public abstract class AbstractEventAccess implements EventAccess {
activeGroupThreads += processScheduler.getActiveThreadCount(funnel);
}
final int statelessActiveThreadCount;
if (group.resolveExecutionEngine() == ExecutionEngine.STATELESS) {
statelessActiveThreadCount = processScheduler.getActiveThreadCount(group);
activeGroupThreads = statelessActiveThreadCount;
} else {
statelessActiveThreadCount = 0;
}
status.setActiveThreadCount(activeGroupThreads);
status.setStatelessActiveThreadCount(statelessActiveThreadCount);
status.setTerminatedThreadCount(terminatedGroupThreads);
status.setBytesRead(bytesRead);
status.setBytesWritten(bytesWritten);

View File

@ -79,7 +79,8 @@ public class FlowDifferenceFilters {
}
private static boolean isSensitivePropertyDueToGhosting(final FlowDifference difference, final FlowManager flowManager) {
if (difference.getDifferenceType() != DifferenceType.PROPERTY_SENSITIVITY_CHANGED) {
final DifferenceType differenceType = difference.getDifferenceType();
if (differenceType != DifferenceType.PROPERTY_SENSITIVITY_CHANGED && differenceType != DifferenceType.PROPERTY_ADDED) {
return false;
}

View File

@ -0,0 +1,82 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.scheduling;
import org.junit.jupiter.api.Test;
import java.util.Optional;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertSame;
import static org.junit.jupiter.api.Assertions.assertTrue;
public class TestStandardLifecycleStateManager {
private static final String COMPONENT_ID = "abc";
@Test
public void testGetOrRegisterWithoutReplace() {
final StandardLifecycleStateManager manager = new StandardLifecycleStateManager();
assertFalse(manager.getLifecycleState(COMPONENT_ID).isPresent());
final LifecycleState registered = manager.getOrRegisterLifecycleState(COMPONENT_ID, false, false);
assertNotNull(registered);
final Optional<LifecycleState> stateOptional = manager.getLifecycleState(COMPONENT_ID);
assertTrue(stateOptional.isPresent());
assertSame(registered, stateOptional.get());
assertSame(registered, manager.getOrRegisterLifecycleState(COMPONENT_ID, false, false));
registered.terminate();
assertSame(registered, manager.getOrRegisterLifecycleState(COMPONENT_ID, false, false));
}
@Test
public void testGetOrRegisterReplaceTerminated() {
final StandardLifecycleStateManager manager = new StandardLifecycleStateManager();
final LifecycleState registered = manager.getOrRegisterLifecycleState(COMPONENT_ID, false, false);
assertSame(registered, manager.getOrRegisterLifecycleState(COMPONENT_ID, true, false));
registered.terminate();
final LifecycleState replacement = manager.getOrRegisterLifecycleState(COMPONENT_ID, true, false);
assertNotNull(replacement);
assertNotEquals(registered, replacement);
}
@Test
public void testGetOrRegisterReplaceUnscheduled() {
final StandardLifecycleStateManager manager = new StandardLifecycleStateManager();
final LifecycleState registered = manager.getOrRegisterLifecycleState(COMPONENT_ID, false, false);
assertSame(registered, manager.getOrRegisterLifecycleState(COMPONENT_ID, true, false));
registered.setScheduled(true);
registered.terminate();
assertSame(registered, manager.getOrRegisterLifecycleState(COMPONENT_ID, false, true));
registered.setScheduled(false);
final LifecycleState replacement = manager.getOrRegisterLifecycleState(COMPONENT_ID, false, true);
assertNotNull(replacement);
assertNotEquals(registered, replacement);
}
}

View File

@ -35,7 +35,7 @@ public class TestStandardLogRepository {
public void testLogRepository() {
StandardLogRepository repo = new StandardLogRepository();
MockLogObserver observer = new MockLogObserver();
repo.addObserver("mock", LogLevel.DEBUG, observer);
repo.addObserver(LogLevel.DEBUG, observer);
IOException exception = new IOException("exception");
@ -50,7 +50,7 @@ public class TestStandardLogRepository {
public void testLogRepositoryLogsFirstFlowFileUuid() {
StandardLogRepository repo = new StandardLogRepository();
MockLogObserver observer = new MockLogObserver();
repo.addObserver("mock", LogLevel.DEBUG, observer);
repo.addObserver(LogLevel.DEBUG, observer);
MockFlowFile mockFlowFile = new MockFlowFile(1L);
repo.addLogMessage(LogLevel.INFO, "Testing {} being shown in exception message", new Object[]{mockFlowFile});
@ -62,7 +62,7 @@ public class TestStandardLogRepository {
public void testLogRepositoryDoesntLogMultipleFlowFileUuids() {
StandardLogRepository repo = new StandardLogRepository();
MockLogObserver observer = new MockLogObserver();
repo.addObserver("mock", LogLevel.DEBUG, observer);
repo.addObserver(LogLevel.DEBUG, observer);
MockFlowFile mockFlowFile1 = new MockFlowFile(1L);
MockFlowFile mockFlowFile2 = new MockFlowFile(2L);

View File

@ -69,5 +69,11 @@ language governing permissions and limitations under the License. -->
<groupId>jakarta.xml.bind</groupId>
<artifactId>jakarta.xml.bind-api</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-stateless-api</artifactId>
<version>2.0.0-SNAPSHOT</version>
<scope>compile</scope>
</dependency>
</dependencies>
</project>

View File

@ -40,5 +40,8 @@ public enum ConnectableType {
* Root Group Output Ports and Local Output Ports
*/
OUTPUT_PORT,
FUNNEL
FUNNEL,
STATELESS_GROUP;
}

View File

@ -16,6 +16,8 @@
*/
package org.apache.nifi.connectable;
import org.apache.nifi.components.PortFunction;
public interface Port extends Connectable {
void shutdown();
@ -30,4 +32,8 @@ public interface Port extends Connectable {
void onSchedulingStart();
void disable();
void setPortFunction(PortFunction portFunction);
PortFunction getPortFunction();
}

View File

@ -43,6 +43,8 @@ import org.apache.nifi.controller.service.ControllerServiceDisabledException;
import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.controller.service.ControllerServiceProvider;
import org.apache.nifi.controller.service.ControllerServiceState;
import org.apache.nifi.flow.ExecutionEngine;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.NarCloseable;
import org.apache.nifi.parameter.ExpressionLanguageAgnosticParameterParser;
@ -912,7 +914,17 @@ public abstract class AbstractComponentNode implements ComponentNode {
}
if (!controllerServiceNode.isActive()) {
validationResults.add(new DisabledServiceValidationResult(descriptor.getDisplayName(), controllerServiceId));
// If the component belongs to a Process Group and the Process Group is configured to use the Stateless Engine,
// do not validate that the Controller Service is valid. We do this because we cannot start & stop Controller Services
// individually. But if using the standard NiFi engine, or if the component doesn't belong to a Process Group,
// we want to require that the referenced Controller Service is invalid.
// Additionally, if the Controller Service is in a higher level Process Group that is not using the Stateless Engine, it must be enabled.
final ProcessGroup processGroup = getParentProcessGroup().orElse(null);
final ProcessGroup csGroup = controllerServiceNode.getProcessGroup();
final boolean csGroupStateless = csGroup != null && csGroup.resolveExecutionEngine() == ExecutionEngine.STATELESS;
if (processGroup == null || processGroup.resolveExecutionEngine() == ExecutionEngine.STANDARD || !csGroupStateless) {
validationResults.add(new DisabledServiceValidationResult(descriptor.getDisplayName(), controllerServiceId));
}
} else if (ControllerServiceState.ENABLING == controllerServiceNode.getState()) {
validationResults.add(new EnablingServiceValidationResult(descriptor.getDisplayName(), controllerServiceId));
}

View File

@ -22,6 +22,7 @@ import org.apache.nifi.authorization.Resource;
import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.authorization.resource.ResourceFactory;
import org.apache.nifi.authorization.resource.ResourceType;
import org.apache.nifi.components.PortFunction;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.connectable.ConnectableType;
@ -36,6 +37,8 @@ import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.util.CharacterFilterUtils;
import org.apache.nifi.util.FormatUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Collection;
@ -56,6 +59,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
import static java.util.Objects.requireNonNull;
public abstract class AbstractPort implements Port {
private static final Logger logger = LoggerFactory.getLogger(AbstractPort.class);
public static final Relationship PORT_RELATIONSHIP = new Relationship.Builder()
.description("The relationship through which all Flow Files are transferred")
@ -82,6 +86,7 @@ public abstract class AbstractPort implements Port {
private final AtomicReference<String> versionedComponentId = new AtomicReference<>();
private final AtomicLong schedulingNanos;
private final AtomicLong yieldExpiration;
private final AtomicReference<PortFunction> portFunction = new AtomicReference<>(PortFunction.STANDARD);
private final ProcessScheduler processScheduler;
private final Set<Connection> outgoingConnections;
@ -428,11 +433,13 @@ public abstract class AbstractPort implements Port {
@Override
public void shutdown() {
scheduledState.set(ScheduledState.STOPPED);
logger.info("{} shutdown", this);
}
@Override
public void onSchedulingStart() {
scheduledState.set(ScheduledState.RUNNING);
logger.info("{} started", this);
}
@Override
@ -441,6 +448,7 @@ public abstract class AbstractPort implements Port {
if (!updated) {
throw new IllegalStateException("Port cannot be disabled because it is not stopped");
}
logger.info("{} disabled", this);
}
public void enable() {
@ -448,6 +456,7 @@ public abstract class AbstractPort implements Port {
if (!updated) {
throw new IllegalStateException("Port cannot be enabled because it is not disabled");
}
logger.info("{} enabled", this);
}
@Override
@ -584,7 +593,9 @@ public abstract class AbstractPort implements Port {
final Collection<ValidationResult> validationResults = getValidationErrors();
if (!validationResults.isEmpty()) {
throw new IllegalStateException(this.getIdentifier() + " is not in a valid state: " + validationResults.iterator().next().getExplanation());
final String portType = getConnectableType() == ConnectableType.INPUT_PORT ? "Input Port" : "Output Port";
final String message = String.format("%s %s is not in a valid state: %s", portType, getIdentifier(), validationResults.iterator().next().getExplanation());
throw new IllegalStateException(message);
}
} finally {
readLock.unlock();
@ -717,5 +728,12 @@ public abstract class AbstractPort implements Port {
return value;
}
@Override
public PortFunction getPortFunction() {
return portFunction.get();
}
public void setPortFunction(final PortFunction portFunction) {
this.portFunction.set(requireNonNull(portFunction));
}
}

View File

@ -35,6 +35,7 @@ import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.components.validation.ValidationState;
import org.apache.nifi.components.validation.ValidationStatus;
import org.apache.nifi.context.PropertyContext;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.parameter.ParameterContext;
import org.apache.nifi.parameter.ParameterLookup;
import org.apache.nifi.parameter.ParameterUpdate;
@ -45,6 +46,7 @@ import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.TimeUnit;
@ -278,6 +280,12 @@ public interface ComponentNode extends ComponentAuthorizable {
*/
boolean isSensitiveDynamicProperty(String name);
/**
* @return the Process Group that this component belongs to, or an empty Optional if this component does not belong to a Process Group.
* Note that some components, such as Reporting Tasks, will never belong to a Process Group.
*/
Optional<ProcessGroup> getParentProcessGroup();
@Override
default AuthorizationResult checkAuthorization(Authorizer authorizer, RequestAction action, NiFiUser user, Map<String, String> resourceContext) {
// if this is a modification request and the reporting task is restricted ensure the user has elevated privileges. if this

View File

@ -21,6 +21,7 @@ import org.apache.nifi.connectable.Funnel;
import org.apache.nifi.connectable.Port;
import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.controller.service.ControllerServiceProvider;
import org.apache.nifi.groups.StatelessGroupNode;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.ProcessSessionFactory;
@ -81,7 +82,7 @@ public interface ProcessScheduler {
*
* @param procNode to stop
*/
Future<Void> stopProcessor(ProcessorNode procNode);
CompletableFuture<Void> stopProcessor(ProcessorNode procNode);
/**
* Interrupts all threads that are currently active in the Processor in an attempt to
@ -109,6 +110,20 @@ public interface ProcessScheduler {
*/
void onProcessorRemoved(ProcessorNode procNode);
/**
* Starts scheduling the given Stateless Group to run, after initializing all components.
* @param groupNode the group to start
* @return a Future that will be completed whenever the group has started
*/
Future<Void> startStatelessGroup(StatelessGroupNode groupNode);
/**
* Stops scheduling the given Stateless Group to run. Returns a Future that will be completed whenever all components within the group have been stopped
* @param groupNode the group to stop
* @return a Future that will be completed whenever all components within the group have been stopped
*/
CompletableFuture<Void> stopStatelessGroup(StatelessGroupNode groupNode);
/**
* Notifies the scheduler that the given port has been removed from the flow
*

View File

@ -24,7 +24,6 @@ import org.apache.nifi.components.validation.ValidationTrigger;
import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.controller.scheduling.LifecycleState;
import org.apache.nifi.controller.scheduling.SchedulingAgent;
import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.controller.service.ControllerServiceProvider;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.logging.LogLevel;
@ -129,18 +128,6 @@ public abstract class ProcessorNode extends AbstractComponentNode implements Con
*/
public abstract int getActiveThreadCount();
/**
* Verifies that this Processor can be started if the provided set of
* services are enabled. This is introduced because we need to verify that
* all components can be started before starting any of them. In order to do
* that, we need to know that this component can be started if the given
* services are enabled, as we will then enable the given services before
* starting this component.
*
* @param ignoredReferences to ignore
*/
public abstract void verifyCanStart(Set<ControllerServiceNode> ignoredReferences);
public void verifyCanPerformVerification() {
if (isRunning()) {
throw new IllegalStateException("Cannot perform verification because the Processor is not stopped");
@ -171,6 +158,7 @@ public abstract class ProcessorNode extends AbstractComponentNode implements Con
return sc;
}
/**
* Returns the physical state of this processor which includes transition
* states such as STOPPING and STARTING.
@ -207,9 +195,10 @@ public abstract class ProcessorNode extends AbstractComponentNode implements Con
* then the Processor will automatically restart itself as soon as its last thread finishes. If this
* value is <code>true</code> or if the Processor is in any state other than 'STOPPING' or 'RUNNING', then this method
* will throw an {@link IllegalStateException}.
* @param triggerLifecycleMethods Whether or not the lifecycle methods (@OnScheduled, @OnUnscheduled, @OnStopped, etc.) should be called.
*/
public abstract void start(ScheduledExecutorService scheduler, long administrativeYieldMillis, long timeoutMillis, Supplier<ProcessContext> processContextFactory,
SchedulingAgentCallback schedulingAgentCallback, boolean failIfStopping);
SchedulingAgentCallback schedulingAgentCallback, boolean failIfStopping, boolean triggerLifecycleMethods);
/**
* Will run the {@link Processor} represented by this
@ -251,9 +240,11 @@ public abstract class ProcessorNode extends AbstractComponentNode implements Con
* @param scheduleState
* the ScheduleState that can be used to ensure that the running state (STOPPED, RUNNING, etc.)
* as well as the active thread counts are kept in sync
* @param triggerLifecycleMethods
* whether or not to trigger lifecycle methods such as @OnScheduled, @OnStopped, etc.
*/
public abstract CompletableFuture<Void> stop(ProcessScheduler processScheduler, ScheduledExecutorService executor,
ProcessContext processContext, SchedulingAgent schedulingAgent, LifecycleState scheduleState);
ProcessContext processContext, SchedulingAgent schedulingAgent, LifecycleState scheduleState, boolean triggerLifecycleMethods);
/**
* Marks all active tasks as terminated and interrupts all active threads

View File

@ -32,6 +32,7 @@ import java.util.concurrent.atomic.AtomicInteger;
public class LifecycleState {
private final Object componentId;
private final AtomicInteger activeThreadCount = new AtomicInteger(0);
private final AtomicBoolean scheduled = new AtomicBoolean(false);
private final Set<ScheduledFuture<?>> futures = new HashSet<>();
@ -40,6 +41,10 @@ public class LifecycleState {
private volatile boolean terminated = false;
private final Map<ActiveProcessSessionFactory, Object> activeProcessSessionFactories = new WeakHashMap<>();
public LifecycleState(final String componentId) {
this.componentId = componentId;
}
public synchronized boolean tryIncrementActiveThreadCount(final ActiveProcessSessionFactory sessionFactory) {
if ((terminated) || (!scheduled.get())) {
return false;
@ -103,7 +108,7 @@ public class LifecycleState {
@Override
public String toString() {
return "LifecycleState[activeThreads= " + activeThreadCount.get() + ", scheduled=" + scheduled.get() + "]";
return "LifecycleState[componentId=" + componentId + ", activeThreads= " + activeThreadCount.get() + ", scheduled=" + scheduled.get() + "]";
}
/**
@ -153,4 +158,5 @@ public class LifecycleState {
public boolean isTerminated() {
return this.terminated;
}
}

View File

@ -0,0 +1,52 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.scheduling;
import java.util.Optional;
public interface LifecycleStateManager {
/**
* Returns the LifecycleState that is registered for the given component; if
* no LifecycleState currently is registered, one is created and registered
* atomically, and then that value is returned.
*
* @param componentId the ID of the component that the lifecycle state is associated with
* @param replaceTerminatedState if there exists a LifecycleState for the given component but that component has been terminated,
* indicates whether it should be replaced with a new one or not
* @param replaceUnscheduledState if there exists a LifecycleState for the given component and that component has been unscheduled,
* indicates whether it should be replaced with a new one or not
* @return the lifecycle state for the given schedulable component
*/
LifecycleState getOrRegisterLifecycleState(String componentId, boolean replaceTerminatedState, boolean replaceUnscheduledState);
/**
* Returns the LifecycleState that is registered for the given component; if no LifecycleState
* is registered, returns an empty Optional.
* @param componentId the ID of the component that the lifecycle state is associated with
* @return the lifecycle state associated with the given component, or an empty Optional
*/
Optional<LifecycleState> getLifecycleState(String componentId);
/**
* Removes the LifecycleState for the given component and returns the value that was removed, or an empty Optional
* if there was no state registered
* @param componentId the ID of the component that the lifecycle state is associated with
* @return the removed state or an empty optional
*/
Optional<LifecycleState> removeLifecycleState(String componentId);
}

View File

@ -128,17 +128,28 @@ public interface ControllerServiceNode extends ComponentNode, VersionedComponent
/**
* Indicates that the given component is now referencing this Controller Service
* @param referringComponent the component referencing this service
* @param referencingComponent the component referencing this service
* @param propertyDescriptor the property for which the component is referencing this controller service
*/
void addReference(ComponentNode referringComponent, PropertyDescriptor propertyDescriptor);
void addReference(ComponentNode referencingComponent, PropertyDescriptor propertyDescriptor);
/**
* Updates the component reference for the given component in order to provide the new property descriptor. This may be necessary if
* a property descriptor's definition has changed to reference a new type of service, stop referencing services, or start referencing services.
* This is not typical but may happen for more dynamic components such as those that are defined by a script. If there is currently no reference
* to this service for the given component and property descriptor, a new one will be added.
*
* @param referencingComponent the component referencing this Controller Service
* @param propertyDescriptor the updated property descriptor
*/
void updateReference(ComponentNode referencingComponent, PropertyDescriptor propertyDescriptor);
/**
* Indicates that the given component is no longer referencing this Controller Service
* @param referringComponent the component that is no longer referencing this service
* @param referencingComponent the component that is no longer referencing this service
* @param propertyDescriptor the property for which the component is referencing this controller service
*/
void removeReference(ComponentNode referringComponent, PropertyDescriptor propertyDescriptor);
void removeReference(ComponentNode referencingComponent, PropertyDescriptor propertyDescriptor);
void setComments(String comment);

View File

@ -109,7 +109,7 @@ public interface ControllerServiceProvider extends ControllerServiceLookup {
* @param serviceNodes the nodes the disable
* @return a Future that can be used to cancel the task or wait until it is completed
*/
Future<Void> disableControllerServicesAsync(Collection<ControllerServiceNode> serviceNodes);
CompletableFuture<Void> disableControllerServicesAsync(Collection<ControllerServiceNode> serviceNodes);
/**
* @return a Set of all Controller Services that exist for this service provider

View File

@ -117,23 +117,19 @@ public final class BulletinFactory {
}
private static ComponentType getComponentType(final Connectable connectable) {
final ComponentType type;
switch (connectable.getConnectableType()) {
case REMOTE_INPUT_PORT:
case REMOTE_OUTPUT_PORT:
type = ComponentType.REMOTE_PROCESS_GROUP;
break;
return ComponentType.REMOTE_PROCESS_GROUP;
case INPUT_PORT:
type = ComponentType.INPUT_PORT;
break;
return ComponentType.INPUT_PORT;
case OUTPUT_PORT:
type = ComponentType.OUTPUT_PORT;
break;
return ComponentType.OUTPUT_PORT;
case STATELESS_GROUP:
return ComponentType.PROCESS_GROUP;
case PROCESSOR:
default:
type = ComponentType.PROCESSOR;
break;
return ComponentType.PROCESSOR;
}
return type;
}
}

View File

@ -22,6 +22,7 @@ import org.apache.nifi.connectable.Port;
import org.apache.nifi.controller.ProcessorNode;
import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.controller.service.ControllerServiceProvider;
import org.apache.nifi.flow.ExecutionEngine;
import org.apache.nifi.flow.ScheduledState;
import org.apache.nifi.registry.flow.mapping.VersionedComponentStateLookup;
import org.apache.nifi.remote.RemoteGroupPort;
@ -30,9 +31,12 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Queue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.atomic.AtomicLong;
import java.util.stream.Collectors;
public abstract class AbstractComponentScheduler implements ComponentScheduler {
private static final Logger logger = LoggerFactory.getLogger(AbstractComponentScheduler.class);
@ -44,6 +48,7 @@ public abstract class AbstractComponentScheduler implements ComponentScheduler {
private final Queue<Connectable> connectablesToStart = new LinkedBlockingQueue<>();
private final Queue<ReportingTaskNode> reportingTasksToStart = new LinkedBlockingQueue<>();
private final Queue<ControllerServiceNode> toEnable = new LinkedBlockingQueue<>();
private final Queue<ProcessGroup> statelessGroupsToStart = new LinkedBlockingQueue<>();
public AbstractComponentScheduler(final ControllerServiceProvider controllerServiceProvider, final VersionedComponentStateLookup stateLookup) {
this.serviceProvider = controllerServiceProvider;
@ -79,6 +84,12 @@ public abstract class AbstractComponentScheduler implements ComponentScheduler {
logger.debug("{} starting {}", this, taskNode);
startNow(taskNode);
}
ProcessGroup processGroup;
while ((processGroup = statelessGroupsToStart.poll()) != null) {
logger.debug("{} starting {}", this, processGroup);
startNow(processGroup);
}
}
private boolean isPaused() {
@ -246,9 +257,50 @@ public abstract class AbstractComponentScheduler implements ComponentScheduler {
}
}
@Override
public void startStatelessGroup(final ProcessGroup group) {
if (isPaused()) {
logger.debug("{} called to start {} but paused so will queue it for start later", this, group);
statelessGroupsToStart.offer(group);
} else {
logger.debug("{} starting {} now", this, group);
startNow(group);
}
}
@Override
public void stopStatelessGroup(final ProcessGroup group) {
group.stopProcessing();
}
protected abstract void startNow(Connectable component);
protected abstract void enableNow(Collection<ControllerServiceNode> controllerServices);
protected abstract void startNow(ReportingTaskNode reportingTask);
protected abstract void startNow(ProcessGroup statelessGroup);
protected void enableNow(final Collection<ControllerServiceNode> controllerServices) {
final Map<ExecutionEngine, List<ControllerServiceNode>> servicesByExecutionEngine = controllerServices.stream()
.collect(Collectors.groupingBy(this::getExecutionEngine));
final List<ControllerServiceNode> statelessServices = servicesByExecutionEngine.get(ExecutionEngine.STATELESS);
if (statelessServices != null && !statelessServices.isEmpty()) {
logger.debug("Will not enable {} Controller Services because their Execution Engine is Stateless: {}", statelessServices.size(), statelessServices);
}
final List<ControllerServiceNode> servicesToEnable = servicesByExecutionEngine.get(ExecutionEngine.STANDARD);
if (servicesToEnable != null && !servicesToEnable.isEmpty()) {
getControllerServiceProvider().enableControllerServices(servicesToEnable);
}
}
private ExecutionEngine getExecutionEngine(final ControllerServiceNode service) {
final ProcessGroup group = service.getProcessGroup();
if (group == null) {
return ExecutionEngine.STANDARD;
}
return group.resolveExecutionEngine();
}
}

View File

@ -41,6 +41,10 @@ public interface ComponentScheduler {
void resume();
void startStatelessGroup(ProcessGroup group);
void stopStatelessGroup(ProcessGroup group);
ComponentScheduler NOP_SCHEDULER = new ComponentScheduler() {
@Override
@ -73,5 +77,13 @@ public interface ComponentScheduler {
@Override
public void resume() {
}
@Override
public void startStatelessGroup(final ProcessGroup group) {
}
@Override
public void stopStatelessGroup(final ProcessGroup group) {
}
};
}

View File

@ -35,6 +35,7 @@ import org.apache.nifi.controller.label.Label;
import org.apache.nifi.controller.queue.DropFlowFileStatus;
import org.apache.nifi.controller.queue.QueueSize;
import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.flow.ExecutionEngine;
import org.apache.nifi.flow.VersionedExternalFlow;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.parameter.ParameterContext;
@ -48,8 +49,10 @@ import org.apache.nifi.remote.RemoteGroupPort;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Future;
import java.util.function.Predicate;
@ -156,15 +159,59 @@ public interface ProcessGroup extends ComponentAuthorizable, Positionable, Versi
/**
* Starts all Processors, Local Ports, and Funnels that are directly within
* this group and any child ProcessGroups, except for those that are
* disabled.
* disabled. If the ProcessGroup is configured to use the Stateless Engine
* then the state is updated accordingly.
*/
void startProcessing();
/**
* Starts all Processors, Local Ports, and Funnels that are directly within
* this group and any child ProcessGroups, except for those that are
* disabled.
*/
void startComponents();
/**
* Stops all components and if configured to use the Stateless Engine updates state accordingly
*/
CompletableFuture<Void> stopProcessing();
/**
* Stops all Processors, Local Ports, and Funnels that are directly within
* this group and child ProcessGroups, except for those that are disabled.
*/
void stopProcessing();
CompletableFuture<Void> stopComponents();
/**
* @return the scheduled state for this ProcessGroup, or StatelessGroupScheduledState.STOPPED
* if the ProcessGroup is not configured to use the Stateless Engine
*/
StatelessGroupScheduledState getStatelessScheduledState();
/**
* @return the desired state of the group. This is in contrast to the current scheduled state that is returned by
* {@link #getStatelessScheduledState()} when the group is starting or stopping
*/
StatelessGroupScheduledState getDesiredStatelessScheduledState();
/**
* @return <code>true</code> if the ProcessGroup is configured to use the Stateless Engine and it is actively running,
* <code>false</code> if it is stopped and there are no active threads
*/
boolean isStatelessActive();
/**
* Sets the Execution Engine to the given value
* @param executionEngine the execution engine that should be used for running the Process Group
*/
void setExecutionEngine(ExecutionEngine executionEngine);
/**
* Returns the StatelessGroupNode that is used for running this Process Group via the Stateless Engine
*
* @return the StatelessGroupNode or an empty Optional if not running via the Stateless Engine
*/
Optional<StatelessGroupNode> getStatelessGroupNode();
/**
* Enables the given Processor
@ -189,11 +236,6 @@ public interface ProcessGroup extends ComponentAuthorizable, Positionable, Versi
*/
void enableOutputPort(Port port);
/**
* Recursively enables all Controller Services for this Process Group and all child Process Groups
*
*/
void enableAllControllerServices();
/**
* Starts the given Processor
@ -244,7 +286,7 @@ public interface ProcessGroup extends ComponentAuthorizable, Positionable, Versi
*
* @param processor to stop
*/
Future<Void> stopProcessor(ProcessorNode processor);
CompletableFuture<Void> stopProcessor(ProcessorNode processor);
/**
* Terminates the given Processor
@ -930,6 +972,8 @@ public interface ProcessGroup extends ComponentAuthorizable, Positionable, Versi
void verifyCanStart();
void verifyCanScheduleComponentsIndividually();
void verifyCanStop(Connectable connectable);
void verifyCanStop();
@ -1245,4 +1289,48 @@ public interface ProcessGroup extends ComponentAuthorizable, Positionable, Versi
* @param logFileSuffix new log file suffix
*/
void setLogFileSuffix(String logFileSuffix);
/*
* @return the configured Execution Engine for this Process Group, or an Empty Optional if none is configured
*/
ExecutionEngine getExecutionEngine();
/**
* Returns the Execution Engine that should be used for this Process Group. If the Process Group has an Execution Engine explicitly configured,
* it will be returned. Otherwise, the Execution Engine will be resolved by traversing up the Process Group hierarchy until an Execution Engine
* is found.
*
* @return the Execution Engine that should be used for this Process Group
*/
ExecutionEngine resolveExecutionEngine();
/**
* Verifies that the Execution Engine can be set to the configured value.
* @param executionEngine the Execution Engine to set
* @throws IllegalStateException if the Execution Engine cannot be set to the given value
*/
void verifyCanSetExecutionEngine(ExecutionEngine executionEngine);
/**
* Sets the maximum number on concurrent tasks that can be run in this Process Group if using the Stateless Execution Engine
* @param maxConcurrentTasks the maximum number of concurrent tasks
*/
void setMaxConcurrentTasks(int maxConcurrentTasks);
/**
* @return the maximum number of concurrent tasks that can be run in this Process Group if using the Stateless Execution Engine
*/
int getMaxConcurrentTasks();
/**
* Sets the maximum amount of time that a Stateless Flow can run before it times out and is considered a failure.
* @param timeout a string representation of the timeout
*/
void setStatelessFlowTimeout(String timeout);
/**
* Returns the configured maximum amount of time that a Stateless Flow can run before it times out and is considered a failure
* @return the configured maximum amount of time that a Stateless Flow can run before it times out and is considered a failure
*/
String getStatelessFlowTimeout();
}

View File

@ -0,0 +1,40 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.groups;
import org.apache.nifi.flow.VersionedExternalFlow;
public interface StatelessGroupFactory {
/**
* Creates a VersionedExternalFlow from the given ProcessGroup
* @param group the ProcessGroup to create the VersionedExternalFlow from
* @return the VersionedExternalFlow
*/
VersionedExternalFlow createVersionedExternalFlow(ProcessGroup group);
/**
* Creates a ProcessGroup that can be used as the root group for a Stateless flow, based on the given VersionedExternalFlow and existing ProcessGroup.
* The given group will then be set as the parent of the returned ProcessGroup, in order to facilitate the normal hierarchy of ProcessGroups, including access to
* Controller Services that are defined at a higher level, etc.
*
* @param group the ProcessGroup
* @param versionedExternalFlow the VersionedExternalFlow that was created from the given Process Group
* @return the ProcessGroup that can be used as the root group for a Stateless flow
*/
ProcessGroup createStatelessProcessGroup(ProcessGroup group, VersionedExternalFlow versionedExternalFlow);
}

View File

@ -0,0 +1,47 @@
/*
* 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.groups;
import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.controller.ProcessScheduler;
import org.apache.nifi.controller.ScheduledState;
import org.apache.nifi.controller.SchedulingAgentCallback;
import org.apache.nifi.controller.scheduling.LifecycleState;
import org.apache.nifi.controller.scheduling.SchedulingAgent;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
public interface StatelessGroupNode extends Connectable {
void initialize(StatelessGroupNodeInitializationContext initializationContext);
ProcessGroup getProcessGroup();
void start(ScheduledExecutorService executorService, SchedulingAgentCallback schedulingAgentCallback, LifecycleState lifecycleState);
CompletableFuture<Void> stop(ProcessScheduler processScheduler, ScheduledExecutorService executor, SchedulingAgent schedulingAgent, LifecycleState scheduleState);
ScheduledState getCurrentState();
ScheduledState getDesiredState();
long getBoredYieldDuration(TimeUnit timeUnit);
}

View File

@ -0,0 +1,24 @@
/*
* 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.groups;
import org.apache.nifi.logging.ComponentLog;
public interface StatelessGroupNodeInitializationContext {
ComponentLog getLogger();
}

View File

@ -0,0 +1,23 @@
/*
* 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.groups;
public enum StatelessGroupScheduledState {
RUNNING,
STOPPED;
}

View File

@ -28,35 +28,18 @@ public interface LogRepository {
* Registers an observer so that it will be notified of all Log Messages
* whose levels are at least equal to the given level.
*
* @param observerIdentifier identifier of observer
* @param level of logs the observer wants
* @param observer the observer
*/
void addObserver(String observerIdentifier, LogLevel level, LogObserver observer);
void addObserver(LogLevel level, LogObserver observer);
/**
* Sets the observation level of the specified observer.
*
* @param observerIdentifier identifier of observer
* @param level of logs the observer wants
*/
void setObservationLevel(String observerIdentifier, LogLevel level);
void setObservationLevel(LogLevel level);
/**
* Gets the observation level for the specified observer.
*
* @param observerIdentifier identifier of observer
* @return level
*/
LogLevel getObservationLevel(String observerIdentifier);
/**
* Removes the given LogObserver from this Repository.
*
* @param observerIdentifier identifier of observer
* @return old log observer
*/
LogObserver removeObserver(String observerIdentifier);
/**
* Removes all LogObservers from this Repository

View File

@ -22,6 +22,8 @@ import org.apache.nifi.controller.ProcessorNode;
import org.apache.nifi.controller.ReportingTaskNode;
import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.flow.ScheduledState;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.groups.StatelessGroupScheduledState;
public interface VersionedComponentStateLookup {
ScheduledState getState(ProcessorNode processorNode);
@ -32,6 +34,9 @@ public interface VersionedComponentStateLookup {
ScheduledState getState(ControllerServiceNode serviceNode);
ScheduledState getState(ProcessGroup processGroup);
/**
* Returns a Scheduled State of ENABLED or DISABLED for every component. No component will be mapped to RUNNING.
*/
@ -55,6 +60,11 @@ public interface VersionedComponentStateLookup {
public ScheduledState getState(final ControllerServiceNode serviceNode) {
return ScheduledState.DISABLED;
}
@Override
public ScheduledState getState(final ProcessGroup group) {
return ScheduledState.ENABLED;
}
};
/**
@ -89,6 +99,15 @@ public interface VersionedComponentStateLookup {
}
}
@Override
public ScheduledState getState(final ProcessGroup group) {
if (group.getDesiredStatelessScheduledState() == StatelessGroupScheduledState.RUNNING) {
return ScheduledState.RUNNING;
}
return ScheduledState.ENABLED;
}
private ScheduledState map(final org.apache.nifi.controller.ScheduledState componentState) {
if (componentState == null) {
return null;

View File

@ -33,6 +33,7 @@ import org.apache.nifi.components.validation.ValidationTrigger;
import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.controller.service.ControllerServiceProvider;
import org.apache.nifi.controller.service.ControllerServiceState;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.parameter.Parameter;
import org.apache.nifi.parameter.ParameterContext;
@ -445,6 +446,11 @@ public class TestAbstractComponentNode {
return true;
}
@Override
public Optional<ProcessGroup> getParentProcessGroup() {
return Optional.empty();
}
@Override
public ParameterLookup getParameterLookup() {
return ParameterLookup.EMPTY;

View File

@ -146,6 +146,11 @@
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-administration</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-stateless-engine</artifactId>
<version>2.0.0-SNAPSHOT</version>
</dependency>
<dependency>
<groupId>com.h2database</groupId>
<artifactId>h2</artifactId>
@ -305,6 +310,12 @@
<artifactId>caffeine</artifactId>
<scope>compile</scope>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-stateless-api</artifactId>
<version>2.0.0-SNAPSHOT</version>
<scope>compile</scope>
</dependency>
</dependencies>
<build>
<plugins>

View File

@ -100,8 +100,10 @@ import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
import org.apache.nifi.controller.repository.claim.StandardContentClaim;
import org.apache.nifi.controller.repository.claim.StandardResourceClaimManager;
import org.apache.nifi.controller.repository.io.LimitedInputStream;
import org.apache.nifi.controller.scheduling.LifecycleStateManager;
import org.apache.nifi.controller.scheduling.QuartzSchedulingAgent;
import org.apache.nifi.controller.scheduling.RepositoryContextFactory;
import org.apache.nifi.controller.scheduling.StandardLifecycleStateManager;
import org.apache.nifi.controller.scheduling.StandardProcessScheduler;
import org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent;
import org.apache.nifi.controller.serialization.FlowSerializationException;
@ -113,9 +115,9 @@ import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.controller.service.ControllerServiceProvider;
import org.apache.nifi.controller.service.ControllerServiceResolver;
import org.apache.nifi.controller.service.StandardConfigurationContext;
import org.apache.nifi.controller.service.StandardControllerServiceApiLookup;
import org.apache.nifi.controller.service.StandardControllerServiceProvider;
import org.apache.nifi.controller.service.StandardControllerServiceResolver;
import org.apache.nifi.controller.service.StandardControllerServiceApiLookup;
import org.apache.nifi.controller.state.manager.StandardStateManagerProvider;
import org.apache.nifi.controller.state.server.ZooKeeperStateServer;
import org.apache.nifi.controller.status.NodeStatus;
@ -147,7 +149,7 @@ import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.groups.BundleUpdateStrategy;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.groups.RemoteProcessGroup;
import org.apache.nifi.groups.StandardProcessGroup;
import org.apache.nifi.groups.StatelessGroupScheduledState;
import org.apache.nifi.nar.ExtensionDefinition;
import org.apache.nifi.nar.ExtensionDiscoveringManager;
import org.apache.nifi.nar.ExtensionManager;
@ -177,7 +179,6 @@ import org.apache.nifi.python.PythonProcessConfig;
import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.registry.flow.mapping.NiFiRegistryFlowMapper;
import org.apache.nifi.registry.flow.mapping.VersionedComponentStateLookup;
import org.apache.nifi.registry.variable.MutableVariableRegistry;
import org.apache.nifi.remote.HttpRemoteSiteListener;
import org.apache.nifi.remote.RemoteGroupPort;
import org.apache.nifi.remote.RemoteResourceManager;
@ -233,6 +234,7 @@ import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ScheduledExecutorService;
@ -274,6 +276,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
private final FlowFileEventRepository flowFileEventRepository;
private final ProvenanceRepository provenanceRepository;
private final BulletinRepository bulletinRepository;
private final LifecycleStateManager lifecycleStateManager;
private final StandardProcessScheduler processScheduler;
private final SnippetManager snippetManager;
private final long gracefulShutdownSeconds;
@ -314,6 +317,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
private final Boolean isSiteToSiteSecure;
private final Set<Connectable> startConnectablesAfterInitialization;
private final Set<ProcessGroup> startGroupsAfterInitialization;
private final Set<RemoteGroupPort> startRemoteGroupPortsAfterInitialization;
private final LeaderElectionManager leaderElectionManager;
private final ClusterCoordinator clusterCoordinator;
@ -543,7 +547,8 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
throw new RuntimeException(e);
}
processScheduler = new StandardProcessScheduler(timerDrivenEngineRef.get(), this, stateManagerProvider, this.nifiProperties);
lifecycleStateManager = new StandardLifecycleStateManager();
processScheduler = new StandardProcessScheduler(timerDrivenEngineRef.get(), this, stateManagerProvider, this.nifiProperties, lifecycleStateManager);
parameterContextManager = new StandardParameterContextManager();
repositoryContextFactory = new RepositoryContextFactory(contentRepository, flowFileRepository, flowFileEventRepository, counterRepositoryRef.get(), provenanceRepository, stateManagerProvider);
@ -576,6 +581,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
startConnectablesAfterInitialization = new HashSet<>();
startRemoteGroupPortsAfterInitialization = new HashSet<>();
startGroupsAfterInitialization = new HashSet<>();
final String gracefulShutdownSecondsVal = nifiProperties.getProperty(GRACEFUL_SHUTDOWN_PERIOD);
long shutdownSecs;
@ -593,18 +599,12 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
remoteInputHttpPort = nifiProperties.getRemoteInputHttpPort();
isSiteToSiteSecure = nifiProperties.isSiteToSiteSecure();
if (isSiteToSiteSecure && sslContext == null && remoteInputSocketPort != null) {
throw new IllegalStateException("NiFi Configured to allow Secure Site-to-Site communications but the Keystore/Truststore properties are not configured");
}
this.heartbeatDelaySeconds = (int) FormatUtils.getTimeDuration(nifiProperties.getNodeHeartbeatInterval(), TimeUnit.SECONDS);
this.snippetManager = new SnippetManager();
this.reloadComponent = new StandardReloadComponent(this);
final ProcessGroup rootGroup = new StandardProcessGroup(ComponentIdGenerator.generateId().toString(), controllerServiceProvider, processScheduler,
encryptor, extensionManager, stateManagerProvider, flowManager, reloadComponent, new MutableVariableRegistry(this.variableRegistry), this,
nifiProperties);
final ProcessGroup rootGroup = flowManager.createProcessGroup(ComponentIdGenerator.generateId().toString());
rootGroup.setName(FlowManager.DEFAULT_ROOT_GROUP_NAME);
setRootGroup(rootGroup);
instanceId = ComponentIdGenerator.generateId().toString();
@ -979,6 +979,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
try {
startConnectablesAfterInitialization.clear();
startRemoteGroupPortsAfterInitialization.clear();
startGroupsAfterInitialization.clear();
} finally {
writeLock.unlock("purge");
}
@ -1163,6 +1164,12 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
validationThreadPool.scheduleWithFixedDelay(new TriggerValidationTask(flowManager, validationTrigger), 5, 5, TimeUnit.SECONDS);
if (startDelayedComponents) {
LOG.info("Starting {} Stateless Process Groups", startGroupsAfterInitialization.size());
for (final ProcessGroup group : startGroupsAfterInitialization) {
group.startProcessing();
}
startGroupsAfterInitialization.clear();
LOG.info("Starting {} processors/ports/funnels", startConnectablesAfterInitialization.size() + startRemoteGroupPortsAfterInitialization.size());
for (final Connectable connectable : startConnectablesAfterInitialization) {
if (connectable.getScheduledState() == ScheduledState.DISABLED) {
@ -1191,7 +1198,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
remoteGroupPort.getRemoteProcessGroup().startTransmitting(remoteGroupPort);
startedTransmitting++;
} catch (final Throwable t) {
LOG.error("Unable to start transmitting with {} due to {}", new Object[]{remoteGroupPort, t});
LOG.error("Unable to start transmitting with {}", remoteGroupPort, t);
}
}
@ -1206,7 +1213,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
startConnectable(connectable);
}
} catch (final Throwable t) {
LOG.error("Unable to start {} due to {}", new Object[]{connectable, t});
LOG.error("Unable to start {}", connectable, t);
}
}
@ -1262,6 +1269,10 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
return startConnectablesAfterInitialization.contains(component) || startRemoteGroupPortsAfterInitialization.contains(component);
}
public boolean isStartAfterInitialization(final ProcessGroup group) {
return startGroupsAfterInitialization.contains(group);
}
private ContentRepository createContentRepository(final NiFiProperties properties) {
final String implementationClassName = isEncryptionProtocolVersionConfigured(properties)
? ENCRYPTED_CONTENT_REPO_IMPLEMENTATION
@ -1353,6 +1364,9 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
return bulletinRepository;
}
public LifecycleStateManager getLifecycleStateManager() {
return lifecycleStateManager;
}
public SnippetManager getSnippetManager() {
return snippetManager;
}
@ -1393,7 +1407,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
public void shutdown(final boolean kill) {
LOG.info("Initiating shutdown of FlowController...");
this.shutdown = true;
flowManager.getRootGroup().stopProcessing();
final CompletableFuture<Void> rootGroupStopFuture = flowManager.getRootGroup().stopProcessing();
readLock.lock();
try {
@ -1409,14 +1423,6 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
heartbeatMonitor.stop();
}
if (kill) {
this.timerDrivenEngineRef.get().shutdownNow();
LOG.info("Initiated immediate shutdown of flow controller...");
} else {
this.timerDrivenEngineRef.get().shutdown();
LOG.info("Initiated graceful shutdown of flow controller...waiting up to " + gracefulShutdownSeconds + " seconds");
}
validationThreadPool.shutdown();
clusterTaskExecutor.shutdownNow();
@ -1444,8 +1450,28 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
processScheduler.shutdownReportingTask(taskNode);
}
final long shutdownEnd = System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(gracefulShutdownSeconds);
if (!kill) {
try {
rootGroupStopFuture.get(gracefulShutdownSeconds, TimeUnit.SECONDS);
} catch (final Exception e) {
LOG.warn("Failed to wait until all components have gracefully stopped", e);
}
}
if (kill) {
this.timerDrivenEngineRef.get().shutdownNow();
LOG.info("Initiated immediate shutdown of flow controller...");
} else {
this.timerDrivenEngineRef.get().shutdown();
LOG.info("Initiated graceful shutdown of flow controller...waiting up to " + gracefulShutdownSeconds + " seconds");
}
try {
this.timerDrivenEngineRef.get().awaitTermination(gracefulShutdownSeconds, TimeUnit.SECONDS);
// Give thread pool up to the configured amount of time to finish, but no less than 2 seconds,
// in order to allow for a more graceful shutdown.
final long millisToWait = Math.max(2000, shutdownEnd - System.currentTimeMillis());
this.timerDrivenEngineRef.get().awaitTermination(millisToWait, TimeUnit.MILLISECONDS);
} catch (final InterruptedException ie) {
LOG.info("Interrupted while waiting for controller termination.");
}
@ -1556,6 +1582,15 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
return port.getScheduledState();
}
@Override
public ScheduledState getScheduledState(final ProcessGroup processGroup) {
if (startGroupsAfterInitialization.contains(processGroup)) {
return ScheduledState.RUNNING;
}
return processGroup.getDesiredStatelessScheduledState() == StatelessGroupScheduledState.RUNNING ? ScheduledState.RUNNING : ScheduledState.STOPPED;
}
};
}
@ -1596,6 +1631,15 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
public org.apache.nifi.flow.ScheduledState getState(final ControllerServiceNode serviceNode) {
return delegate.getState(serviceNode);
}
@Override
public org.apache.nifi.flow.ScheduledState getState(final ProcessGroup group) {
if (isStartAfterInitialization(group)) {
return org.apache.nifi.flow.ScheduledState.RUNNING;
}
return delegate.getState(group);
}
};
}
@ -1734,7 +1778,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
public SystemDiagnostics getSystemDiagnostics() {
final SystemDiagnosticsFactory factory = new SystemDiagnosticsFactory();
return factory.create(flowFileRepository, contentRepository, provenanceRepository);
return factory.create(flowFileRepository, contentRepository, provenanceRepository, resourceClaimManager);
}
public String getContentRepoFileStoreName(final String containerName) {
@ -1924,6 +1968,19 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
}
}
public void startProcessGroup(final ProcessGroup processGroup) {
writeLock.lock();
try {
if (initialized.get()) {
processGroup.startProcessing();
} else {
startGroupsAfterInitialization.add(processGroup);
}
} finally {
writeLock.unlock("startProcessGroup");
}
}
public boolean isInitialized() {
return initialized.get();
}
@ -2032,6 +2089,18 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
startConnectablesAfterInitialization.remove(node);
}
public void stopGroup(final String parentGroupId, final String groupId) {
final ProcessGroup parent = lookupGroup(parentGroupId);
final ProcessGroup group = parent.getProcessGroup(groupId);
if (group == null) {
throw new IllegalStateException("Cannot find ProcessGroup with ID " + groupId + " within ProcessGroup with ID " + parentGroupId);
}
group.stopProcessing();
// If we are ready to start the group upon initialization of the controller, don't.
startGroupsAfterInitialization.remove(group);
}
@Override
public void startReportingTask(final ReportingTaskNode reportingTaskNode) {
@ -2212,13 +2281,15 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
final List<Counter> counters = new ArrayList<>();
final CounterRepository counterRepo = counterRepositoryRef.get();
for (final Counter counter : counterRepo.getCounters()) {
counters.add(counter);
}
counters.addAll(counterRepo.getCounters());
return counters;
}
public CounterRepository getCounterRepository() {
return counterRepositoryRef.get();
}
public Counter resetCounter(final String identifier) {
final CounterRepository counterRepo = counterRepositoryRef.get();
final Counter resetValue = counterRepo.resetCounter(identifier);
@ -3166,6 +3237,10 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
return processScheduler;
}
public long getBoredYieldDuration(final TimeUnit timeUnit) {
return (long) FormatUtils.getPreciseTimeDuration(nifiProperties.getBoredYieldDuration(), timeUnit);
}
public AuditService getAuditService() {
return auditService;
}

View File

@ -30,6 +30,7 @@ import org.apache.nifi.controller.label.Label;
import org.apache.nifi.controller.queue.FlowFileQueue;
import org.apache.nifi.controller.queue.LoadBalanceStrategy;
import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.flow.ExecutionEngine;
import org.apache.nifi.flowfile.FlowFilePrioritizer;
import org.apache.nifi.groups.FlowFileConcurrency;
import org.apache.nifi.groups.FlowFileOutboundPolicy;
@ -495,6 +496,9 @@ public class StandardFlowSnippet implements FlowSnippet {
childGroup.setPosition(toPosition(groupDTO.getPosition()));
childGroup.setComments(groupDTO.getComments());
childGroup.setName(groupDTO.getName());
childGroup.setExecutionEngine(ExecutionEngine.valueOf(groupDTO.getExecutionEngine()));
childGroup.setStatelessFlowTimeout(groupDTO.getStatelessFlowTimeout());
childGroup.setMaxConcurrentTasks(groupDTO.getMaxConcurrentTasks());
final String flowfileConcurrentName = groupDTO.getFlowfileConcurrency();
if (flowfileConcurrentName != null) {

View File

@ -44,7 +44,6 @@ import org.apache.nifi.controller.ProcessorNode;
import org.apache.nifi.controller.ReportingTaskNode;
import org.apache.nifi.controller.StandardFlowSnippet;
import org.apache.nifi.controller.StandardFunnel;
import org.apache.nifi.controller.StandardProcessorNode;
import org.apache.nifi.controller.exception.ComponentLifeCycleException;
import org.apache.nifi.controller.exception.ProcessorInstantiationException;
import org.apache.nifi.controller.label.Label;
@ -60,6 +59,7 @@ import org.apache.nifi.flowfile.FlowFilePrioritizer;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.groups.RemoteProcessGroup;
import org.apache.nifi.groups.StandardProcessGroup;
import org.apache.nifi.groups.StatelessGroupNodeFactory;
import org.apache.nifi.logging.ControllerServiceLogObserver;
import org.apache.nifi.logging.FlowRegistryClientLogObserver;
import org.apache.nifi.logging.LogLevel;
@ -96,6 +96,7 @@ import javax.net.ssl.SSLContext;
import java.net.URL;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@ -125,6 +126,7 @@ public class StandardFlowManager extends AbstractFlowManager implements FlowMana
private final boolean isSiteToSiteSecure;
public StandardFlowManager(final NiFiProperties nifiProperties, final SSLContext sslContext, final FlowController flowController,
final FlowFileEventRepository flowFileEventRepository, final ParameterContextManager parameterContextManager) {
super(flowFileEventRepository, parameterContextManager, flowController::isInitialized);
@ -271,15 +273,17 @@ public class StandardFlowManager extends AbstractFlowManager implements FlowMana
public ProcessGroup createProcessGroup(final String id) {
final MutableVariableRegistry mutableVariableRegistry = new MutableVariableRegistry(flowController.getVariableRegistry());
final StatelessGroupNodeFactory statelessGroupNodeFactory = new StandardStatelessGroupNodeFactory(flowController, sslContext, flowController.createKerberosConfig(nifiProperties));
final ProcessGroup group = new StandardProcessGroup(requireNonNull(id), flowController.getControllerServiceProvider(), processScheduler, flowController.getEncryptor(),
flowController.getExtensionManager(), flowController.getStateManagerProvider(), this,
flowController.getReloadComponent(), mutableVariableRegistry, flowController, nifiProperties);
flowController.getReloadComponent(), mutableVariableRegistry, flowController, nifiProperties, statelessGroupNodeFactory);
onProcessGroupAdded(group);
return group;
}
public void instantiateSnippet(final ProcessGroup group, final FlowSnippetDTO dto) throws ProcessorInstantiationException {
requireNonNull(group);
requireNonNull(dto);
@ -348,7 +352,7 @@ public class StandardFlowManager extends AbstractFlowManager implements FlowMana
LogRepositoryFactory.getRepository(procNode.getIdentifier()).setLogger(procNode.getLogger());
if (registerLogObserver) {
logRepository.addObserver(StandardProcessorNode.BULLETIN_OBSERVER_ID, procNode.getBulletinLevel(), new ProcessorLogObserver(bulletinRepository, procNode));
logRepository.addObserver(procNode.getBulletinLevel(), new ProcessorLogObserver(bulletinRepository, procNode));
}
if (firstTimeAdded) {
@ -358,7 +362,7 @@ public class StandardFlowManager extends AbstractFlowManager implements FlowMana
logDeprecationNotice(processor);
} catch (final Exception e) {
if (registerLogObserver) {
logRepository.removeObserver(StandardProcessorNode.BULLETIN_OBSERVER_ID);
logRepository.removeAllObservers();
}
throw new ComponentLifeCycleException("Failed to invoke @OnAdded methods of " + procNode.getProcessor(), e);
}
@ -433,8 +437,7 @@ public class StandardFlowManager extends AbstractFlowManager implements FlowMana
onFlowRegistryClientAdded(clientNode);
// Register log observer to provide bulletins when reporting task logs anything at WARN level or above
logRepository.addObserver(StandardProcessorNode.BULLETIN_OBSERVER_ID, LogLevel.WARN,
new FlowRegistryClientLogObserver(bulletinRepository, clientNode));
logRepository.addObserver(LogLevel.WARN, new FlowRegistryClientLogObserver(bulletinRepository, clientNode));
}
return clientNode;
@ -512,8 +515,7 @@ public class StandardFlowManager extends AbstractFlowManager implements FlowMana
onReportingTaskAdded(taskNode);
// Register log observer to provide bulletins when reporting task logs anything at WARN level or above
logRepository.addObserver(StandardProcessorNode.BULLETIN_OBSERVER_ID, LogLevel.WARN,
new ReportingTaskLogObserver(bulletinRepository, taskNode));
logRepository.addObserver(LogLevel.WARN, new ReportingTaskLogObserver(bulletinRepository, taskNode));
}
return taskNode;
@ -569,8 +571,7 @@ public class StandardFlowManager extends AbstractFlowManager implements FlowMana
onParameterProviderAdded(parameterProviderNode);
// Register log observer to provide bulletins when reporting task logs anything at WARN level or above
logRepository.addObserver(StandardProcessorNode.BULLETIN_OBSERVER_ID, LogLevel.WARN,
new ParameterProviderLogObserver(bulletinRepository, parameterProviderNode));
logRepository.addObserver(LogLevel.WARN, new ParameterProviderLogObserver(bulletinRepository, parameterProviderNode));
}
return parameterProviderNode;
@ -655,7 +656,7 @@ public class StandardFlowManager extends AbstractFlowManager implements FlowMana
LogRepositoryFactory.getRepository(serviceNode.getIdentifier()).setLogger(serviceNode.getLogger());
if (registerLogObserver) {
logRepository.addObserver(StandardProcessorNode.BULLETIN_OBSERVER_ID, serviceNode.getBulletinLevel(), new ControllerServiceLogObserver(bulletinRepository, serviceNode));
logRepository.addObserver(serviceNode.getBulletinLevel(), new ControllerServiceLogObserver(bulletinRepository, serviceNode));
}
if (firstTimeAdded) {
@ -709,9 +710,7 @@ public class StandardFlowManager extends AbstractFlowManager implements FlowMana
for (final Class<? extends ConfigurableComponent> alternativeClass : deprecationNotice.alternatives()) {
alternatives.add(alternativeClass.getSimpleName());
}
for (final String className : deprecationNotice.classNames()) {
alternatives.add(className);
}
Collections.addAll(alternatives, deprecationNotice.classNames());
deprecationLogger.warn("Added Deprecated Component {}[id={}] See alternatives {}",
componentClass.getSimpleName(),

View File

@ -0,0 +1,316 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.flow;
import org.apache.nifi.bundle.Bundle;
import org.apache.nifi.bundle.BundleCoordinate;
import org.apache.nifi.components.state.StatelessStateManagerProvider;
import org.apache.nifi.controller.FlowController;
import org.apache.nifi.controller.kerberos.KerberosConfig;
import org.apache.nifi.controller.repository.ContentRepository;
import org.apache.nifi.controller.repository.FlowFileEventRepository;
import org.apache.nifi.controller.repository.FlowFileRepository;
import org.apache.nifi.controller.repository.NonPurgeableContentRepository;
import org.apache.nifi.controller.repository.StatelessBridgeFlowFileRepository;
import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
import org.apache.nifi.controller.scheduling.StatelessProcessScheduler;
import org.apache.nifi.controller.scheduling.StatelessProcessSchedulerInitializationContext;
import org.apache.nifi.engine.FlowEngine;
import org.apache.nifi.extensions.BundleAvailability;
import org.apache.nifi.extensions.ExtensionRepository;
import org.apache.nifi.flow.ExternalControllerServiceReference;
import org.apache.nifi.flow.VersionedComponent;
import org.apache.nifi.flow.VersionedExternalFlow;
import org.apache.nifi.flow.VersionedExternalFlowMetadata;
import org.apache.nifi.flow.VersionedParameterContext;
import org.apache.nifi.groups.ComponentIdGenerator;
import org.apache.nifi.groups.ComponentScheduler;
import org.apache.nifi.groups.FlowSynchronizationOptions;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.groups.StandardStatelessGroupNode;
import org.apache.nifi.groups.StatelessGroupFactory;
import org.apache.nifi.groups.StatelessGroupNode;
import org.apache.nifi.groups.StatelessGroupNodeFactory;
import org.apache.nifi.groups.StatelessGroupNodeInitializationContext;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.logging.ConnectableLogObserver;
import org.apache.nifi.logging.LogLevel;
import org.apache.nifi.logging.LogRepository;
import org.apache.nifi.logging.LogRepositoryFactory;
import org.apache.nifi.logging.LoggingContext;
import org.apache.nifi.logging.StandardLoggingContext;
import org.apache.nifi.parameter.ParameterContextManager;
import org.apache.nifi.processor.SimpleProcessLogger;
import org.apache.nifi.provenance.ProvenanceRepository;
import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.registry.flow.mapping.ComponentIdLookup;
import org.apache.nifi.registry.flow.mapping.FlowMappingOptions;
import org.apache.nifi.registry.flow.mapping.InstantiatedVersionedProcessGroup;
import org.apache.nifi.registry.flow.mapping.NiFiRegistryFlowMapper;
import org.apache.nifi.registry.flow.mapping.VersionedComponentStateLookup;
import org.apache.nifi.reporting.BulletinRepository;
import org.apache.nifi.stateless.engine.ProcessContextFactory;
import org.apache.nifi.stateless.engine.StandardStatelessEngine;
import org.apache.nifi.stateless.engine.StatelessEngine;
import org.apache.nifi.stateless.engine.StatelessEngineInitializationContext;
import org.apache.nifi.stateless.engine.StatelessFlowManager;
import org.apache.nifi.stateless.engine.StatelessProcessContextFactory;
import org.apache.nifi.stateless.repository.RepositoryContextFactory;
import org.apache.nifi.stateless.repository.StatelessFlowFileRepository;
import org.apache.nifi.stateless.repository.StatelessProvenanceRepository;
import org.apache.nifi.stateless.repository.StatelessRepositoryContextFactory;
import javax.net.ssl.SSLContext;
import java.time.Duration;
import java.time.temporal.ChronoUnit;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.BooleanSupplier;
import java.util.function.Supplier;
public class StandardStatelessGroupNodeFactory implements StatelessGroupNodeFactory {
private final FlowController flowController;
private final SSLContext sslContext;
private final KerberosConfig kerberosConfig;
private final AtomicReference<FlowEngine> statelessComponentLifecycleThreadPool = new AtomicReference<>();
private final AtomicReference<FlowEngine> statelessComponentMonitoringThreadPool = new AtomicReference<>();
private final AtomicReference<FlowEngine> statelessFrameworkTaskThreadPool = new AtomicReference<>();
public StandardStatelessGroupNodeFactory(final FlowController flowController, final SSLContext sslContext, final KerberosConfig kerberosConfig) {
this.flowController = flowController;
this.sslContext = sslContext;
this.kerberosConfig = kerberosConfig;
}
@Override
public StatelessGroupNode createStatelessGroupNode(final ProcessGroup group) {
// Use a StatelessFlowFile Repository because we want all transient data lost upon restart. We only want to keep the data that's
// completed the flow. However, it's important that when the getNextFlowFileSequence() method is called that we call the method
// on the underlying FlowFile Repository in order to ensure that the FlowFile Sequence is incremented. It's also important that if
// any Content Claims have their claimant count decremented to 0 in the flow and the resource claim is no longer in use that we do
// allow NiFi's FlowFile Repository to handle orchestrating the destruction of the Content Claim. It cannot be done in the Stateless
// flow because execution may be rolled back.
final ResourceClaimManager resourceClaimManager = flowController.getResourceClaimManager();
final BulletinRepository bulletinRepository = flowController.getBulletinRepository();
final FlowFileRepository underlyingFlowFileRepository = flowController.getRepositoryContextFactory().getFlowFileRepository();
final StatelessFlowFileRepository flowFileRepository = new StatelessBridgeFlowFileRepository(underlyingFlowFileRepository, resourceClaimManager);
final StatelessProvenanceRepository statelessProvenanceRepository = new StatelessProvenanceRepository(1_000);
flowFileRepository.initialize(resourceClaimManager);
final ContentRepository contentRepository = new NonPurgeableContentRepository(flowController.getRepositoryContextFactory().getContentRepository());
final RepositoryContextFactory statelessRepoContextFactory = new StatelessRepositoryContextFactory(
contentRepository,
flowFileRepository,
flowController.getFlowFileEventRepository(),
flowController.getCounterRepository(),
statelessProvenanceRepository,
flowController.getStateManagerProvider());
final FlowMappingOptions flowMappingOptions = new FlowMappingOptions.Builder()
.componentIdLookup(ComponentIdLookup.USE_COMPONENT_ID)
.mapControllerServiceReferencesToVersionedId(false)
.mapInstanceIdentifiers(true)
.mapPropertyDescriptors(false)
.mapSensitiveConfiguration(true)
.sensitiveValueEncryptor(value -> value) // No need to encrypt, since we won't be persisting the flow
.stateLookup(VersionedComponentStateLookup.IDENTITY_LOOKUP)
.build();
final StatelessGroupFactory statelessGroupFactory = new StatelessGroupFactory() {
@Override
public VersionedExternalFlow createVersionedExternalFlow(final ProcessGroup group) {
return StandardStatelessGroupNodeFactory.this.createVersionedExternalFlow(group, flowMappingOptions);
}
@Override
public ProcessGroup createStatelessProcessGroup(final ProcessGroup group, final VersionedExternalFlow versionedExternalFlow) {
return StandardStatelessGroupNodeFactory.this.createStatelessProcessGroup(group, versionedExternalFlow, statelessRepoContextFactory, flowMappingOptions);
}
};
final LogRepository logRepository = LogRepositoryFactory.getRepository(group.getIdentifier());
final StatelessGroupNode statelessGroupNode = new StandardStatelessGroupNode.Builder()
.rootGroup(group)
.controllerServiceProvider(flowController.getControllerServiceProvider())
.extensionManager(flowController.getExtensionManager())
.statelessRepositoryContextFactory(statelessRepoContextFactory)
.nifiFlowFileRepository(underlyingFlowFileRepository)
.nifiContentRepository(contentRepository)
.nifiProvenanceRepository(flowController.getProvenanceRepository())
.flowFileEventRepository(flowController.getFlowFileEventRepository())
.stateManagerProvider(flowController.getStateManagerProvider())
.bulletinRepository(flowController.getBulletinRepository())
.statelessGroupFactory(statelessGroupFactory)
.lifecycleStateManager(flowController.getLifecycleStateManager())
.boredYieldDuration(flowController.getBoredYieldDuration(TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS)
.build();
logRepository.removeAllObservers();
logRepository.addObserver(LogLevel.WARN, new ConnectableLogObserver(bulletinRepository, statelessGroupNode));
final LoggingContext loggingContext = new StandardLoggingContext(statelessGroupNode);
final ComponentLog componentLog = new SimpleProcessLogger(statelessGroupNode, logRepository, loggingContext);
final StatelessGroupNodeInitializationContext initContext = () -> componentLog;
statelessGroupNode.initialize(initContext);
return statelessGroupNode;
}
private VersionedExternalFlow createVersionedExternalFlow(final ProcessGroup group, final FlowMappingOptions flowMappingOptions) {
final NiFiRegistryFlowMapper flowMapper = new NiFiRegistryFlowMapper(flowController.getExtensionManager(), flowMappingOptions);
final InstantiatedVersionedProcessGroup versionedGroup = flowMapper.mapNonVersionedProcessGroup(group, flowController.getControllerServiceProvider());
final Map<String, VersionedParameterContext> parameterContexts = flowMapper.mapParameterContexts(group, true, new HashMap<>());
final Map<String, ExternalControllerServiceReference> externalControllerServiceReferences =
Optional.ofNullable(versionedGroup.getExternalControllerServiceReferences()).orElse(Collections.emptyMap());
final VersionedExternalFlow versionedExternalFlow = new VersionedExternalFlow();
versionedExternalFlow.setFlowContents(versionedGroup);
versionedExternalFlow.setExternalControllerServices(externalControllerServiceReferences);
versionedExternalFlow.setParameterContexts(parameterContexts);
versionedExternalFlow.setParameterProviders(Collections.emptyMap());
final VersionedExternalFlowMetadata metadata = new VersionedExternalFlowMetadata();
metadata.setFlowName(group.getName());
versionedExternalFlow.setMetadata(metadata);
return versionedExternalFlow;
}
private ProcessGroup createStatelessProcessGroup(final ProcessGroup group, final VersionedExternalFlow versionedExternalFlow, final RepositoryContextFactory statelessRepositoryContextFactory,
final FlowMappingOptions flowMappingOptions) {
final FlowFileEventRepository flowFileEventRepository = flowController.getFlowFileEventRepository();
final BooleanSupplier flowInitializedCheck = flowController::isInitialized;
final ParameterContextManager parameterContextManager = flowController.getFlowManager().getParameterContextManager();
final ExtensionRepository extensionRepository = new ExtensionRepository() {
@Override
public void initialize() {
}
@Override
public BundleAvailability getBundleAvailability(final BundleCoordinate bundleCoordinate) {
return BundleAvailability.BUNDLE_AVAILABLE;
}
@Override
public Future<Set<Bundle>> fetch(final Set<BundleCoordinate> bundleCoordinates, final ExecutorService executorService, final int concurrentDownloads) {
return CompletableFuture.completedFuture(Collections.emptySet());
}
};
final StatelessProcessScheduler statelessScheduler = new StatelessProcessScheduler(flowController.getExtensionManager(), Duration.of(10, ChronoUnit.SECONDS));
final StatelessStateManagerProvider stateManagerProvider = new StatelessStateManagerProvider();
final StatelessEngine statelessEngine = new StandardStatelessEngine.Builder()
.bulletinRepository(flowController.getBulletinRepository())
.counterRepository(flowController.getCounterRepository())
.encryptor(flowController.getEncryptor())
.extensionManager(flowController.getExtensionManager())
.extensionRepository(extensionRepository)
.flowFileEventRepository(flowFileEventRepository)
.processScheduler(statelessScheduler)
.provenanceRepository((ProvenanceRepository) statelessRepositoryContextFactory.getProvenanceRepository())
.stateManagerProvider(stateManagerProvider)
.kerberosConfiguration(kerberosConfig)
.statusTaskInterval(null)
.variableRegistry(VariableRegistry.EMPTY_REGISTRY)
.build();
final BulletinRepository bulletinRepository = flowController.getBulletinRepository();
final StatelessFlowManager statelessFlowManager = new StatelessFlowManager(flowFileEventRepository, parameterContextManager, statelessEngine, flowInitializedCheck,
sslContext, bulletinRepository);
final ProcessContextFactory processContextFactory = new StatelessProcessContextFactory(flowController.getControllerServiceProvider(), stateManagerProvider);
final StatelessEngineInitializationContext engineInitContext = new StatelessEngineInitializationContext(flowController.getControllerServiceProvider(), statelessFlowManager,
processContextFactory, statelessRepositoryContextFactory);
statelessEngine.initialize(engineInitContext);
final FlowEngine componentLifecycleThreadPool = lazyInitializeThreadPool(statelessComponentLifecycleThreadPool,
() -> new FlowEngine(8, "Stateless Component Lifecycle", true));
final FlowEngine componentMonitoringThreadPool = lazyInitializeThreadPool(statelessComponentMonitoringThreadPool,
() -> new FlowEngine(2, "Stateless Component Monitoring", true));
final FlowEngine frameworkTaskThreadPool = lazyInitializeThreadPool(statelessFrameworkTaskThreadPool,
() -> new FlowEngine(2, "Stateless Framework Tasks", true));
final StatelessProcessSchedulerInitializationContext schedulerInitializationContext = new StatelessProcessSchedulerInitializationContext.Builder()
.processContextFactory(processContextFactory)
.componentLifeCycleThreadPool(componentLifecycleThreadPool)
.componentMonitoringThreadPool(componentMonitoringThreadPool)
.frameworkTaskThreadPool(frameworkTaskThreadPool)
.manageThreadPools(false)
.build();
statelessScheduler.initialize(schedulerInitializationContext);
final ProcessGroup tempRootGroup = statelessFlowManager.createProcessGroup("root");
tempRootGroup.setName("root");
statelessFlowManager.setRootGroup(tempRootGroup);
final ProcessGroup child = statelessFlowManager.createProcessGroup(group.getIdentifier());
child.setName(group.getName());
child.setParent(tempRootGroup);
final ComponentIdGenerator idGenerator = (proposedId, instanceId, destinationGroupId) -> instanceId;
final FlowSynchronizationOptions synchronizationOptions = new FlowSynchronizationOptions.Builder()
.componentComparisonIdLookup(VersionedComponent::getInstanceIdentifier)
.componentIdGenerator(idGenerator)
.componentScheduler(ComponentScheduler.NOP_SCHEDULER)
.componentStopTimeout(Duration.ofSeconds(60))
.propertyDecryptor(value -> value)
.topLevelGroupId(group.getIdentifier())
.updateDescendantVersionedFlows(true)
.updateGroupSettings(true)
.updateGroupVersionControlSnapshot(false)
.updateRpgUrls(true)
.ignoreLocalModifications(true)
.build();
child.synchronizeFlow(versionedExternalFlow, synchronizationOptions, flowMappingOptions);
child.setParent(group);
return child;
}
private FlowEngine lazyInitializeThreadPool(final AtomicReference<FlowEngine> reference, final Supplier<FlowEngine> factory) {
FlowEngine threadPool = reference.get();
if (threadPool == null) {
threadPool = factory.get();
final boolean updated = reference.compareAndSet(null, threadPool);
if (!updated) {
threadPool.shutdown();
threadPool = reference.get();
}
}
return threadPool;
}
}

View File

@ -747,6 +747,7 @@ public class FileSystemRepository implements ContentRepository {
// Ensure that we have no writable claim streams for this resource claim
final ByteCountingOutputStream bcos = writableClaimStreams.remove(claim);
LOG.debug("Removed Stream {} for {} from writableClaimStreams because Resource Claim was removed", bcos, claim);
if (bcos != null) {
try {
@ -1171,6 +1172,7 @@ public class FileSystemRepository implements ContentRepository {
// claimant count is removed without writing to the claim (or more specifically, without closing the
// OutputStream that is returned when calling write() ).
final OutputStream out = writableClaimStreams.remove(claim);
LOG.debug("Removed {} for {} from writableClaimStreams because Resource Claim was archived", out, claim);
if (out != null) {
try {
@ -1908,6 +1910,10 @@ public class FileSystemRepository implements ContentRepository {
@Override
public synchronized void close() throws IOException {
if (closed) {
return;
}
closed = true;
if (alwaysSync) {
@ -1939,8 +1945,9 @@ public class FileSystemRepository implements ContentRepository {
if (enqueued) {
LOG.debug("Claim length less than max; Adding {} back to Writable Claim Queue", this);
} else {
writableClaimStreams.remove(scc.getResourceClaim());
final OutputStream out = writableClaimStreams.remove(scc.getResourceClaim());
resourceClaimManager.freeze(scc.getResourceClaim());
LOG.debug("Removed {} for {} from writableClaimStreams because ContentRepositoryOutputStream was closed and could not enqueue.", out, scc.getResourceClaim());
bcos.close();

View File

@ -0,0 +1,187 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.repository;
import org.apache.nifi.controller.repository.claim.ContentClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaim;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.file.Path;
import java.util.Collection;
import java.util.Set;
public class NonPurgeableContentRepository implements ContentRepository {
private final ContentRepository delegate;
public NonPurgeableContentRepository(final ContentRepository delegate) {
this.delegate = delegate;
}
@Override
public void initialize(final ContentRepositoryContext context) throws IOException {
// No-op
}
@Override
public void shutdown() {
// No-op
}
@Override
public Set<String> getContainerNames() {
return delegate.getContainerNames();
}
@Override
public long getContainerCapacity(final String containerName) throws IOException {
return delegate.getContainerCapacity(containerName);
}
@Override
public long getContainerUsableSpace(final String containerName) throws IOException {
return delegate.getContainerUsableSpace(containerName);
}
@Override
public String getContainerFileStoreName(final String containerName) {
return delegate.getContainerFileStoreName(containerName);
}
@Override
public ContentClaim create(final boolean lossTolerant) throws IOException {
return delegate.create(lossTolerant);
}
@Override
public int incrementClaimaintCount(final ContentClaim claim) {
return delegate.incrementClaimaintCount(claim);
}
@Override
public int getClaimantCount(final ContentClaim claim) {
return delegate.getClaimantCount(claim);
}
@Override
public int decrementClaimantCount(final ContentClaim claim) {
return delegate.decrementClaimantCount(claim);
}
@Override
public boolean remove(final ContentClaim claim) {
return delegate.remove(claim);
}
@Override
public ContentClaim clone(final ContentClaim original, final boolean lossTolerant) throws IOException {
return delegate.clone(original, lossTolerant);
}
@Override
@Deprecated
public long merge(final Collection<ContentClaim> claims, final ContentClaim destination, final byte[] header, final byte[] footer, final byte[] demarcator) throws IOException {
return delegate.merge(claims, destination, header, footer, demarcator);
}
@Override
public long importFrom(final Path content, final ContentClaim claim) throws IOException {
return delegate.importFrom(content, claim);
}
@Override
public long importFrom(final InputStream content, final ContentClaim claim) throws IOException {
return delegate.importFrom(content, claim);
}
@Override
public long exportTo(final ContentClaim claim, final Path destination, final boolean append) throws IOException {
return delegate.exportTo(claim, destination, append);
}
@Override
public long exportTo(final ContentClaim claim, final Path destination, final boolean append, final long offset, final long length) throws IOException {
return delegate.exportTo(claim, destination, append, offset, length);
}
@Override
public long exportTo(final ContentClaim claim, final OutputStream destination) throws IOException {
return delegate.exportTo(claim, destination);
}
@Override
public long exportTo(final ContentClaim claim, final OutputStream destination, final long offset, final long length) throws IOException {
return delegate.exportTo(claim, destination, offset, length);
}
@Override
public long size(final ContentClaim claim) throws IOException {
return delegate.size(claim);
}
@Override
public long size(final ResourceClaim claim) throws IOException {
return delegate.size(claim);
}
@Override
public InputStream read(final ContentClaim claim) throws IOException {
return delegate.read(claim);
}
@Override
public InputStream read(final ResourceClaim claim) throws IOException {
return delegate.read(claim);
}
@Override
public boolean isResourceClaimStreamSupported() {
return delegate.isResourceClaimStreamSupported();
}
@Override
public OutputStream write(final ContentClaim claim) throws IOException {
return delegate.write(claim);
}
@Override
public void purge() {
// No-op
}
@Override
public void cleanup() {
// No-op
}
@Override
public boolean isAccessible(final ContentClaim contentClaim) throws IOException {
return delegate.isAccessible(contentClaim);
}
@Override
public Set<ResourceClaim> getActiveResourceClaims(final String containerName) throws IOException {
return delegate.getActiveResourceClaims(containerName);
}
@Override
public boolean isActiveResourceClaimsSupported() {
return delegate.isActiveResourceClaimsSupported();
}
}

View File

@ -0,0 +1,89 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.repository;
import org.apache.nifi.controller.repository.claim.ContentClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
import org.apache.nifi.stateless.repository.StatelessFlowFileRepository;
import java.io.IOException;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.Set;
/**
* A FlowFile Repository that bridges between the Stateless FlowFile Repository and the NiFi instance's FlowFile Repository
*/
public class StatelessBridgeFlowFileRepository extends StatelessFlowFileRepository {
private final FlowFileRepository nifiFlowFileRepository;
private final ResourceClaimManager resourceClaimManager;
public StatelessBridgeFlowFileRepository(final FlowFileRepository nifiFlowFileRepository, final ResourceClaimManager resourceClaimManager) {
this.nifiFlowFileRepository = nifiFlowFileRepository;
this.resourceClaimManager = resourceClaimManager;
}
@Override
public long getNextFlowFileSequence() {
return nifiFlowFileRepository.getNextFlowFileSequence();
}
@Override
public void updateRepository(final Collection<RepositoryRecord> records) throws IOException {
super.updateRepository(records);
final Set<ContentClaim> transientClaims = new HashSet<>();
for (final RepositoryRecord record : records) {
final RepositoryRecordType type = record.getType();
if (type == RepositoryRecordType.DELETE || type == RepositoryRecordType.CONTENTMISSING) {
final ContentClaim contentClaim = record.getCurrentClaim();
if (isDestructable(contentClaim)) {
transientClaims.add(contentClaim);
}
}
if (record.isContentModified()) {
final ContentClaim contentClaim = record.getOriginalClaim();
if (isDestructable(contentClaim)) {
transientClaims.add(contentClaim);
}
}
}
if (!transientClaims.isEmpty()) {
nifiFlowFileRepository.updateRepository(Collections.singletonList(new StandardRepositoryRecord(transientClaims)));
}
}
private boolean isDestructable(final ContentClaim contentClaim) {
if (contentClaim == null) {
return false;
}
final ResourceClaim resourceClaim = contentClaim.getResourceClaim();
if (resourceClaim.isInUse()) {
return false;
}
final int claimantCount = resourceClaimManager.getClaimantCount(resourceClaim);
return claimantCount == 0;
}
}

View File

@ -542,7 +542,7 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
}
if (!claimsToAdd.isEmpty()) {
// Get / Register a Set<ContentClaim> for the given Partiton Index
// Get / Register a Set<ContentClaim> for the given Partition Index
final Integer partitionKey = Integer.valueOf(partitionIndex);
BlockingQueue<ResourceClaim> claimQueue = claimsAwaitingDestruction.get(partitionKey);
if (claimQueue == null) {

View File

@ -55,21 +55,21 @@ abstract class AbstractSchedulingAgent implements SchedulingAgent {
}
@Override
public void unschedule(Connectable connectable, LifecycleState scheduleState) {
scheduleState.setScheduled(false);
this.doUnschedule(connectable, scheduleState);
public void unschedule(Connectable connectable, LifecycleState lifeycleState) {
lifeycleState.setScheduled(false);
this.doUnschedule(connectable, lifeycleState);
}
@Override
public void schedule(ReportingTaskNode taskNode, LifecycleState scheduleState) {
scheduleState.setScheduled(true);
this.doSchedule(taskNode, scheduleState);
public void schedule(ReportingTaskNode taskNode, LifecycleState lifecycleState) {
lifecycleState.setScheduled(true);
this.doSchedule(taskNode, lifecycleState);
}
@Override
public void unschedule(ReportingTaskNode taskNode, LifecycleState scheduleState) {
scheduleState.setScheduled(false);
this.doUnschedule(taskNode, scheduleState);
public void unschedule(ReportingTaskNode taskNode, LifecycleState lifecycleState) {
lifecycleState.setScheduled(false);
this.doUnschedule(taskNode, lifecycleState);
}
/**

View File

@ -21,6 +21,7 @@ import org.apache.nifi.annotation.lifecycle.OnShutdown;
import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
import org.apache.nifi.annotation.notification.PrimaryNodeState;
import org.apache.nifi.authorization.resource.ComponentAuthorizable;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.components.state.StateManagerProvider;
import org.apache.nifi.components.validation.ValidationStatus;
@ -42,6 +43,8 @@ import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.controller.service.ControllerServiceProvider;
import org.apache.nifi.controller.service.StandardConfigurationContext;
import org.apache.nifi.engine.FlowEngine;
import org.apache.nifi.flow.ExecutionEngine;
import org.apache.nifi.groups.StatelessGroupNode;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.logging.StandardLoggingContext;
import org.apache.nifi.nar.NarCloseable;
@ -65,7 +68,9 @@ import java.util.concurrent.Callable;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.function.Supplier;
@ -84,8 +89,8 @@ public final class StandardProcessScheduler implements ProcessScheduler {
private final String administrativeYieldDuration;
private final StateManagerProvider stateManagerProvider;
private final long processorStartTimeoutMillis;
private final LifecycleStateManager lifecycleStateManager;
private final ConcurrentMap<Object, LifecycleState> lifecycleStates = new ConcurrentHashMap<>();
private final ScheduledExecutorService frameworkTaskExecutor;
private final ConcurrentMap<SchedulingStrategy, SchedulingAgent> strategyAgentMap = new ConcurrentHashMap<>();
@ -94,10 +99,12 @@ public final class StandardProcessScheduler implements ProcessScheduler {
private final ScheduledExecutorService componentMonitoringThreadPool = new FlowEngine(2, "Monitor Processor Lifecycle", true);
public StandardProcessScheduler(final FlowEngine componentLifecycleThreadPool, final FlowController flowController,
final StateManagerProvider stateManagerProvider, final NiFiProperties nifiProperties) {
final StateManagerProvider stateManagerProvider, final NiFiProperties nifiProperties,
final LifecycleStateManager lifecycleStateManager) {
this.componentLifeCycleThreadPool = componentLifecycleThreadPool;
this.flowController = flowController;
this.stateManagerProvider = stateManagerProvider;
this.lifecycleStateManager = lifecycleStateManager;
administrativeYieldDuration = nifiProperties.getAdministrativeYieldDuration();
administrativeYieldMillis = FormatUtils.getTimeDuration(administrativeYieldDuration, TimeUnit.MILLISECONDS);
@ -197,7 +204,7 @@ public final class StandardProcessScheduler implements ProcessScheduler {
@Override
public void schedule(final ReportingTaskNode taskNode) {
final LifecycleState lifecycleState = getLifecycleState(requireNonNull(taskNode), true);
final LifecycleState lifecycleState = getLifecycleState(requireNonNull(taskNode).getIdentifier(), true, false);
if (lifecycleState.isScheduled()) {
return;
}
@ -269,7 +276,7 @@ public final class StandardProcessScheduler implements ProcessScheduler {
@Override
public Future<Void> unschedule(final ReportingTaskNode taskNode) {
final LifecycleState lifecycleState = getLifecycleState(requireNonNull(taskNode), false);
final LifecycleState lifecycleState = getLifecycleState(requireNonNull(taskNode).getIdentifier(), false, false);
if (!lifecycleState.isScheduled()) {
return CompletableFuture.completedFuture(null);
}
@ -326,24 +333,31 @@ public final class StandardProcessScheduler implements ProcessScheduler {
/**
* Starts the given {@link Processor} by invoking its
* {@link ProcessorNode#start(ScheduledExecutorService, long, long, Supplier, SchedulingAgentCallback, boolean)}
* {@link ProcessorNode#start(ScheduledExecutorService, long, long, Supplier, SchedulingAgentCallback, boolean, boolean)}
* method.
*
* @see StandardProcessorNode#start(ScheduledExecutorService, long, long, Supplier, SchedulingAgentCallback, boolean)
* @see StandardProcessorNode#start(ScheduledExecutorService, long, long, Supplier, SchedulingAgentCallback, boolean, boolean)
*/
@Override
public synchronized CompletableFuture<Void> startProcessor(final ProcessorNode procNode, final boolean failIfStopping) {
final LifecycleState lifecycleState = getLifecycleState(requireNonNull(procNode), true);
final LifecycleState lifecycleState = getLifecycleState(requireNonNull(procNode), true, false);
final Supplier<ProcessContext> processContextFactory = () -> new StandardProcessContext(procNode, getControllerServiceProvider(),
getStateManager(procNode.getIdentifier()), lifecycleState::isTerminated, flowController);
final boolean scheduleActions = procNode.getProcessGroup().resolveExecutionEngine() != ExecutionEngine.STATELESS;
final CompletableFuture<Void> future = new CompletableFuture<>();
final SchedulingAgentCallback callback = new SchedulingAgentCallback() {
@Override
public void trigger() {
lifecycleState.clearTerminationFlag();
getSchedulingAgent(procNode).schedule(procNode, lifecycleState);
// If using stateless engine, no need to schedule the component to run within the standard NiFi scheduler.
if (scheduleActions) {
getSchedulingAgent(procNode).schedule(procNode, lifecycleState);
}
future.complete(null);
}
@ -360,7 +374,44 @@ public final class StandardProcessScheduler implements ProcessScheduler {
};
LOG.info("Starting {}", procNode);
procNode.start(componentMonitoringThreadPool, administrativeYieldMillis, processorStartTimeoutMillis, processContextFactory, callback, failIfStopping);
procNode.start(componentMonitoringThreadPool, administrativeYieldMillis, processorStartTimeoutMillis, processContextFactory, callback, failIfStopping, scheduleActions);
return future;
}
public synchronized CompletableFuture<Void> startStatelessGroup(final StatelessGroupNode groupNode) {
final LifecycleState lifecycleState = getLifecycleState(requireNonNull(groupNode), true, true);
lifecycleState.setScheduled(true);
final CompletableFuture<Void> future = new CompletableFuture<>();
final SchedulingAgentCallback callback = new SchedulingAgentCallback() {
@Override
public void onTaskComplete() {
}
@Override
public Future<?> scheduleTask(final Callable<?> task) {
return componentLifeCycleThreadPool.submit(task);
}
@Override
public void trigger() {
lifecycleState.clearTerminationFlag();
// Start each Processor. This won't trigger the Processor to run because the Execution Engine will be Stateless.
// However, it will transition its scheduled state to the appropriate value.
groupNode.getProcessGroup().findAllProcessors().forEach(proc -> startProcessor(proc, false));
groupNode.getProcessGroup().findAllInputPorts().forEach(port -> startConnectable(port));
groupNode.getProcessGroup().findAllOutputPorts().forEach(port -> startConnectable(port));
groupNode.getProcessGroup().findAllControllerServices().forEach(service -> enableControllerService(service));
getSchedulingAgent(groupNode).schedule(groupNode, lifecycleState);
future.complete(null);
}
};
LOG.info("Starting {}", groupNode);
groupNode.start(componentMonitoringThreadPool, callback, lifecycleState);
return future;
}
@ -373,7 +424,7 @@ public final class StandardProcessScheduler implements ProcessScheduler {
*/
@Override
public Future<Void> runProcessorOnce(ProcessorNode procNode, final Callable<Future<Void>> stopCallback) {
final LifecycleState lifecycleState = getLifecycleState(requireNonNull(procNode), true);
final LifecycleState lifecycleState = getLifecycleState(requireNonNull(procNode), true, false);
final Supplier<ProcessContext> processContextFactory = () -> new StandardProcessContext(procNode, getControllerServiceProvider(),
getStateManager(procNode.getIdentifier()), lifecycleState::isTerminated, flowController);
@ -407,29 +458,37 @@ public final class StandardProcessScheduler implements ProcessScheduler {
/**
* Stops the given {@link Processor} by invoking its
* {@link ProcessorNode#stop(ProcessScheduler, ScheduledExecutorService, ProcessContext, SchedulingAgent, LifecycleState)}
* {@link ProcessorNode#stop(ProcessScheduler, ScheduledExecutorService, ProcessContext, SchedulingAgent, LifecycleState, boolean)}
* method.
*
* @see StandardProcessorNode#stop(ProcessScheduler, ScheduledExecutorService, ProcessContext, SchedulingAgent, LifecycleState)
* @see StandardProcessorNode#stop(ProcessScheduler, ScheduledExecutorService, ProcessContext, SchedulingAgent, LifecycleState, boolean)
*/
@Override
public synchronized CompletableFuture<Void> stopProcessor(final ProcessorNode procNode) {
final LifecycleState lifecycleState = getLifecycleState(procNode, false);
final LifecycleState lifecycleState = getLifecycleState(procNode, false, false);
StandardProcessContext processContext = new StandardProcessContext(procNode, getControllerServiceProvider(),
getStateManager(procNode.getIdentifier()), lifecycleState::isTerminated, flowController);
final boolean triggerLifecycleMethods = procNode.getProcessGroup().resolveExecutionEngine() != ExecutionEngine.STATELESS;
LOG.info("Stopping {}", procNode);
return procNode.stop(this, this.componentLifeCycleThreadPool, processContext, getSchedulingAgent(procNode), lifecycleState);
return procNode.stop(this, this.componentLifeCycleThreadPool, processContext, getSchedulingAgent(procNode), lifecycleState, triggerLifecycleMethods);
}
@Override
public CompletableFuture<Void> stopStatelessGroup(final StatelessGroupNode groupNode) {
final LifecycleState lifecycleState = getLifecycleState(groupNode, false, false);
return groupNode.stop(this, this.componentLifeCycleThreadPool, getSchedulingAgent(groupNode), lifecycleState);
}
@Override
public synchronized void terminateProcessor(final ProcessorNode procNode) {
if (procNode.getScheduledState() != ScheduledState.STOPPED && procNode.getScheduledState() != ScheduledState.RUN_ONCE) {
throw new IllegalStateException("Cannot terminate " + procNode + " because it is not currently stopped");
}
final LifecycleState state = getLifecycleState(procNode, false);
final LifecycleState state = getLifecycleState(procNode, false, false);
if (state.getActiveThreadCount() == 0) {
LOG.debug("Will not terminate {} because it has no active threads", procNode);
return;
@ -455,13 +514,13 @@ public final class StandardProcessScheduler implements ProcessScheduler {
@Override
public void notifyPrimaryNodeStateChange(final ProcessorNode processor, final PrimaryNodeState primaryNodeState) {
final LifecycleState lifecycleState = getLifecycleState(processor, false);
final LifecycleState lifecycleState = getLifecycleState(processor, false, false);
processor.notifyPrimaryNodeChanged(primaryNodeState, lifecycleState);
}
@Override
public void notifyPrimaryNodeStateChange(final ReportingTaskNode taskNode, final PrimaryNodeState primaryNodeState) {
final LifecycleState lifecycleState = getLifecycleState(taskNode, false);
final LifecycleState lifecycleState = getLifecycleState(taskNode.getIdentifier(), false, false);
taskNode.notifyPrimaryNodeChanged(primaryNodeState, lifecycleState);
}
@ -472,22 +531,22 @@ public final class StandardProcessScheduler implements ProcessScheduler {
@Override
public void onProcessorRemoved(final ProcessorNode procNode) {
lifecycleStates.remove(procNode);
lifecycleStateManager.removeLifecycleState(procNode.getIdentifier());
}
@Override
public void onPortRemoved(final Port port) {
lifecycleStates.remove(port);
lifecycleStateManager.removeLifecycleState(port.getIdentifier());
}
@Override
public void onFunnelRemoved(Funnel funnel) {
lifecycleStates.remove(funnel);
lifecycleStateManager.removeLifecycleState(funnel.getIdentifier());
}
@Override
public void onReportingTaskRemoved(final ReportingTaskNode reportingTask) {
lifecycleStates.remove(reportingTask);
lifecycleStateManager.removeLifecycleState(reportingTask.getIdentifier());
}
@Override
@ -518,7 +577,8 @@ public final class StandardProcessScheduler implements ProcessScheduler {
@Override
public int getActiveThreadCount(final Object scheduled) {
return getLifecycleState(scheduled, false).getActiveThreadCount();
final String componentId = getComponentId(scheduled);
return getLifecycleState(componentId, false, false).getActiveThreadCount();
}
@Override
@ -554,7 +614,7 @@ public final class StandardProcessScheduler implements ProcessScheduler {
throw new IllegalStateException(connectable.getIdentifier() + " is disabled, so it cannot be started");
}
final LifecycleState lifecycleState = getLifecycleState(requireNonNull(connectable), true);
final LifecycleState lifecycleState = getLifecycleState(requireNonNull(connectable), true, false);
if (lifecycleState.isScheduled()) {
return;
}
@ -565,12 +625,18 @@ public final class StandardProcessScheduler implements ProcessScheduler {
}
lifecycleState.clearTerminationFlag();
getSchedulingAgent(connectable).schedule(connectable, lifecycleState);
// Schedule the component to be triggered, unless the engine is stateless. For stateless engine, we let the stateless
// framework take care of triggering components.
if (connectable.getProcessGroup().resolveExecutionEngine() != ExecutionEngine.STATELESS) {
getSchedulingAgent(connectable).schedule(connectable, lifecycleState);
}
lifecycleState.setScheduled(true);
}
private synchronized void stopConnectable(final Connectable connectable) {
final LifecycleState state = getLifecycleState(requireNonNull(connectable), false);
final LifecycleState state = getLifecycleState(requireNonNull(connectable), false, false);
if (!state.isScheduled()) {
return;
}
@ -657,8 +723,17 @@ public final class StandardProcessScheduler implements ProcessScheduler {
@Override
public boolean isScheduled(final Object scheduled) {
final LifecycleState lifecycleState = lifecycleStates.get(scheduled);
return lifecycleState == null ? false : lifecycleState.isScheduled();
final String componentId = getComponentId(scheduled);
final LifecycleState lifecycleState = lifecycleStateManager.getLifecycleState(componentId).orElse(null);
return lifecycleState != null && lifecycleState.isScheduled();
}
private String getComponentId(final Object scheduled) {
if (scheduled instanceof ComponentAuthorizable) {
return ((ComponentAuthorizable) scheduled).getIdentifier();
}
return null;
}
/**
@ -666,39 +741,15 @@ public final class StandardProcessScheduler implements ProcessScheduler {
* no LifecycleState current is registered, one is created and registered
* atomically, and then that value is returned.
*
* @param schedulable schedulable
* @param connectable the component
* @return scheduled state
*/
private LifecycleState getLifecycleState(final Object schedulable, final boolean replaceTerminatedState) {
LifecycleState lifecycleState;
while (true) {
lifecycleState = this.lifecycleStates.get(schedulable);
private LifecycleState getLifecycleState(final Connectable connectable, final boolean replaceTerminatedState, final boolean replaceUnscheduledState) {
return getLifecycleState(connectable.getIdentifier(), replaceTerminatedState, replaceUnscheduledState);
}
if (lifecycleState == null) {
lifecycleState = new LifecycleState();
final LifecycleState existing = this.lifecycleStates.putIfAbsent(schedulable, lifecycleState);
if (existing == null) {
break;
} else {
continue;
}
} else if (replaceTerminatedState && lifecycleState.isTerminated()) {
final LifecycleState newLifecycleState = new LifecycleState();
final boolean replaced = this.lifecycleStates.replace(schedulable, lifecycleState, newLifecycleState);
if (replaced) {
lifecycleState = newLifecycleState;
break;
} else {
continue;
}
} else {
break;
}
}
return lifecycleState;
private LifecycleState getLifecycleState(final String componentId, final boolean replaceTerminatedState, final boolean replaceUnscheduledState) {
return lifecycleStateManager.getOrRegisterLifecycleState(componentId, replaceTerminatedState, replaceUnscheduledState);
}
@Override
@ -710,7 +761,28 @@ public final class StandardProcessScheduler implements ProcessScheduler {
@Override
public CompletableFuture<Void> disableControllerService(final ControllerServiceNode service) {
LOG.info("Disabling {}", service);
return service.disable(this.componentLifeCycleThreadPool);
// Because of the shutdown lifecycle, we may need to disable controller services even after the
// thread pool has been shutdown. If this happens, we will use a new thread pool specifically for this
// task and then immediately shut it down. Otherwise, use the existing thread pool
if (componentLifeCycleThreadPool.isShutdown() || componentLifeCycleThreadPool.isTerminated()) {
return disableControllerServiceWithStandaloneThreadPool(service);
} else {
try {
return service.disable(this.componentLifeCycleThreadPool);
} catch (final RejectedExecutionException ree) {
return disableControllerServiceWithStandaloneThreadPool(service);
}
}
}
private CompletableFuture<Void> disableControllerServiceWithStandaloneThreadPool(final ControllerServiceNode service) {
final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor();
try {
return service.disable(executor);
} finally {
executor.shutdown();
}
}
@Override
@ -720,15 +792,13 @@ public final class StandardProcessScheduler implements ProcessScheduler {
}
CompletableFuture<Void> future = null;
if (!requireNonNull(services).isEmpty()) {
for (ControllerServiceNode controllerServiceNode : services) {
final CompletableFuture<Void> serviceFuture = this.disableControllerService(controllerServiceNode);
for (ControllerServiceNode controllerServiceNode : services) {
final CompletableFuture<Void> serviceFuture = this.disableControllerService(controllerServiceNode);
if (future == null) {
future = serviceFuture;
} else {
future = CompletableFuture.allOf(future, serviceFuture);
}
if (future == null) {
future = serviceFuture;
} else {
future = CompletableFuture.allOf(future, serviceFuture);
}
}

View File

@ -162,8 +162,8 @@ public class TimerDrivenSchedulingAgent extends AbstractTimeBasedSchedulingAgent
}
@Override
public void doUnschedule(final Connectable connectable, final LifecycleState scheduleState) {
for (final ScheduledFuture<?> future : scheduleState.getFutures()) {
public void doUnschedule(final Connectable connectable, final LifecycleState lifecycleState) {
for (final ScheduledFuture<?> future : lifecycleState.getFutures()) {
// stop scheduling to run but do not interrupt currently running tasks.
future.cancel(false);
}
@ -172,8 +172,8 @@ public class TimerDrivenSchedulingAgent extends AbstractTimeBasedSchedulingAgent
}
@Override
public void doUnschedule(final ReportingTaskNode taskNode, final LifecycleState scheduleState) {
for (final ScheduledFuture<?> future : scheduleState.getFutures()) {
public void doUnschedule(final ReportingTaskNode taskNode, final LifecycleState lifecycleState) {
for (final ScheduledFuture<?> future : lifecycleState.getFutures()) {
// stop scheduling to run but do not interrupt currently running tasks.
future.cancel(false);
}

View File

@ -35,10 +35,12 @@ import org.apache.nifi.controller.service.ControllerServiceState;
import org.apache.nifi.flow.ComponentType;
import org.apache.nifi.flow.ConnectableComponent;
import org.apache.nifi.flow.ConnectableComponentType;
import org.apache.nifi.flow.ExecutionEngine;
import org.apache.nifi.flow.VersionedComponent;
import org.apache.nifi.flow.VersionedConnection;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.groups.RemoteProcessGroup;
import org.apache.nifi.groups.StatelessGroupScheduledState;
import org.apache.nifi.parameter.ParameterContext;
import org.apache.nifi.registry.flow.FlowRegistryClientNode;
import org.apache.nifi.registry.flow.diff.DifferenceType;
@ -82,6 +84,7 @@ public class AffectedComponentSet {
private final Set<ReportingTaskNode> reportingTasks = new HashSet<>();
private final Set<ParameterProviderNode> parameterProviders = new HashSet<>();
private final Set<FlowRegistryClientNode> flowRegistryClients = new HashSet<>();
private final Set<ProcessGroup> statelessProcessGroups = new HashSet<>();
public AffectedComponentSet(final FlowController flowController) {
this.flowController = flowController;
@ -93,7 +96,12 @@ public class AffectedComponentSet {
return;
}
inputPorts.add(port);
final ProcessGroup statelessGroup = getStatelessGroup(port.getProcessGroup());
if (statelessGroup == null) {
inputPorts.add(port);
} else {
statelessProcessGroups.add(statelessGroup);
}
}
public void addOutputPort(final Port port) {
@ -101,7 +109,12 @@ public class AffectedComponentSet {
return;
}
outputPorts.add(port);
final ProcessGroup statelessGroup = getStatelessGroup(port.getProcessGroup());
if (statelessGroup == null) {
outputPorts.add(port);
} else {
statelessProcessGroups.add(statelessGroup);
}
}
public void addRemoteInputPort(final RemoteGroupPort port) {
@ -109,7 +122,12 @@ public class AffectedComponentSet {
return;
}
remoteInputPorts.add(port);
final ProcessGroup statelessGroup = getStatelessGroup(port.getProcessGroup());
if (statelessGroup == null) {
remoteInputPorts.add(port);
} else {
statelessProcessGroups.add(statelessGroup);
}
}
public void addRemoteOutputPort(final RemoteGroupPort port) {
@ -117,7 +135,12 @@ public class AffectedComponentSet {
return;
}
remoteOutputPorts.add(port);
final ProcessGroup statelessGroup = getStatelessGroup(port.getProcessGroup());
if (statelessGroup == null) {
remoteOutputPorts.add(port);
} else {
statelessProcessGroups.add(statelessGroup);
}
}
public void addRemoteProcessGroup(final RemoteProcessGroup remoteProcessGroup) {
@ -125,8 +148,13 @@ public class AffectedComponentSet {
return;
}
remoteProcessGroup.getInputPorts().forEach(this::addRemoteInputPort);
remoteProcessGroup.getOutputPorts().forEach(this::addRemoteOutputPort);
final ProcessGroup statelessGroup = getStatelessGroup(remoteProcessGroup.getProcessGroup());
if (statelessGroup == null) {
remoteProcessGroup.getInputPorts().forEach(this::addRemoteInputPort);
remoteProcessGroup.getOutputPorts().forEach(this::addRemoteOutputPort);
} else {
statelessProcessGroups.add(statelessGroup);
}
}
public void addProcessor(final ProcessorNode processor) {
@ -134,7 +162,25 @@ public class AffectedComponentSet {
return;
}
processors.add(processor);
final ProcessGroup statelessGroup = getStatelessGroup(processor.getProcessGroup());
if (statelessGroup == null) {
processors.add(processor);
} else {
statelessProcessGroups.add(statelessGroup);
}
}
public void addStatelessGroup(final ProcessGroup group) {
if (group == null) {
return;
}
final ProcessGroup statelessGroup = getStatelessGroup(group);
if (statelessGroup == null) {
return;
}
statelessProcessGroups.add(statelessGroup);
}
public void addControllerService(final ControllerServiceNode controllerService) {
@ -142,6 +188,12 @@ public class AffectedComponentSet {
return;
}
final ProcessGroup statelessGroup = getStatelessGroup(controllerService.getProcessGroup());
if (statelessGroup != null) {
statelessProcessGroups.add(statelessGroup);
return;
}
controllerServices.add(controllerService);
final List<ComponentNode> referencingComponents = controllerService.getReferences().findRecursiveReferences(ComponentNode.class);
@ -359,6 +411,12 @@ public class AffectedComponentSet {
final Set<ProcessGroup> boundGroups = context.getParameterReferenceManager().getProcessGroupsBound(context);
for (final ProcessGroup group : boundGroups) {
final ProcessGroup statelessGroup = getStatelessGroup(group);
if (statelessGroup != null) {
statelessProcessGroups.add(statelessGroup);
continue;
}
group.getProcessors().stream()
.filter(AbstractComponentNode::isReferencingParameter)
.forEach(this::addProcessor);
@ -377,6 +435,12 @@ public class AffectedComponentSet {
return;
}
final ProcessGroup statelessGroup = getStatelessGroup(group);
if (statelessGroup != null) {
statelessProcessGroups.add(statelessGroup);
return;
}
group.getProcessors().stream()
.filter(AbstractComponentNode::isReferencingParameter)
.forEach(this::addProcessor);
@ -386,6 +450,23 @@ public class AffectedComponentSet {
.forEach(this::addControllerService);
}
private ProcessGroup getStatelessGroup(final ProcessGroup start) {
if (start == null) {
return null;
}
final ExecutionEngine executionEngine = start.getExecutionEngine();
switch (executionEngine) {
case STATELESS:
return start;
case INHERITED:
return getStatelessGroup(start.getParent());
case STANDARD:
default:
return null;
}
}
private void addComponentsForParameterUpdate(final FlowDifference difference) {
final DifferenceType differenceType = difference.getDifferenceType();
@ -458,6 +539,9 @@ public class AffectedComponentSet {
addOutputPort(flowManager.getOutputPort(componentId));
break;
case PROCESS_GROUP:
final ProcessGroup group = flowManager.getGroup(componentId);
final ProcessGroup statelessGroup = getStatelessGroup(group);
addStatelessGroup(statelessGroup);
break;
case PROCESSOR:
addProcessor(flowManager.getProcessorNode(componentId));
@ -506,9 +590,12 @@ public class AffectedComponentSet {
controllerServices.stream().filter(service -> ACTIVE_CONTROLLER_SERVICE_STATES.contains(service.getState()))
.forEach(active::addControllerServiceWithoutReferences);
statelessProcessGroups.stream().filter(group -> group.getStatelessScheduledState() == StatelessGroupScheduledState.RUNNING).forEach(active::addStatelessGroup);
return active;
}
private boolean isActive(final ProcessorNode processor) {
// We consider component active if it's starting, running, or has active threads. The call to ProcessorNode.isRunning() will only return true if it has active threads or a scheduled
// state of RUNNING but not if it has a scheduled state of STARTING. We also consider if the processor is to be started once the flow controller has been fully initialized, as
@ -533,6 +620,7 @@ public class AffectedComponentSet {
remoteOutputPorts.forEach(port -> port.getRemoteProcessGroup().startTransmitting(port));
processors.forEach(processor -> processor.getProcessGroup().startProcessor(processor, false));
reportingTasks.forEach(flowController::startReportingTask);
statelessProcessGroups.forEach(group -> group.startProcessing());
}
public void removeComponents(final ComponentSetFilter filter) {
@ -544,6 +632,7 @@ public class AffectedComponentSet {
controllerServices.removeIf(filter::testControllerService);
reportingTasks.removeIf(filter::testReportingTask);
flowRegistryClients.removeIf(filter::testFlowRegistryClient);
statelessProcessGroups.removeIf(filter::testStatelessGroup);
}
/**
@ -565,6 +654,7 @@ public class AffectedComponentSet {
reportingTasks.stream().filter(task -> flowController.getReportingTaskNode(task.getIdentifier()) != null).forEach(existing::addReportingTask);
controllerServices.stream().filter(service -> serviceProvider.getControllerServiceNode(service.getIdentifier()) != null).forEach(existing::addControllerServiceWithoutReferences);
flowRegistryClients.stream().filter(client -> flowManager.getFlowRegistryClient(client.getIdentifier()) != null).forEach(existing::addFlowRegistryClient);
statelessProcessGroups.stream().filter(group -> flowManager.getGroup(group.getIdentifier()) != null).forEach(existing::addStatelessGroup);
return existing;
}
@ -586,6 +676,7 @@ public class AffectedComponentSet {
processors.stream().filter(this::isStartable).forEach(startable::addProcessor);
reportingTasks.stream().filter(this::isStartable).forEach(startable::addReportingTask);
controllerServices.stream().filter(this::isStartable).forEach(startable::addControllerServiceWithoutReferences);
statelessProcessGroups.stream().filter(this::isStartable).forEach(startable::addStatelessGroup);
return startable;
}
@ -605,6 +696,21 @@ public class AffectedComponentSet {
return true;
}
private boolean isStartable(final ProcessGroup group) {
if (group == null) {
return false;
}
if (group.getExecutionEngine() != ExecutionEngine.STATELESS) {
return false;
}
if (group.getStatelessScheduledState() != StatelessGroupScheduledState.STOPPED) {
return false;
}
return true;
}
private boolean isStartable(final Port port) {
if (port == null) {
return false;
@ -623,6 +729,7 @@ public class AffectedComponentSet {
remoteOutputPorts.forEach(port -> port.getRemoteProcessGroup().stopTransmitting(port));
processors.forEach(processor -> processor.getProcessGroup().stopProcessor(processor));
reportingTasks.forEach(flowController::stopReportingTask);
statelessProcessGroups.forEach(group -> group.stopProcessing());
waitForConnectablesStopped();
@ -662,7 +769,7 @@ public class AffectedComponentSet {
if (count++ % 1000 == 0) {
// The 0th time and every 1000th time (10 seconds), log an update
logger.info("Waiting for all required Processors and Reporting Tasks to stop...");
if (reportingTasks.isEmpty() && processors.isEmpty()) {
if (reportingTasks.isEmpty() && processors.isEmpty() && statelessProcessGroups.isEmpty()) {
return;
}
@ -674,6 +781,11 @@ public class AffectedComponentSet {
.filter(processor -> !isStopped(processor))
.collect(Collectors.toSet());
logger.debug("There are currently {} active Processors: {}", activeProcessors.size(), activeProcessors);
final Set<ProcessGroup> activeStatelessGroups = statelessProcessGroups.stream()
.filter(ProcessGroup::isStatelessActive)
.collect(Collectors.toSet());
logger.debug("There are currently {} active Stateless Groups: {}", activeStatelessGroups.size(), activeStatelessGroups);
}
}
@ -691,6 +803,9 @@ public class AffectedComponentSet {
if (reportingTasks.stream().anyMatch(ReportingTaskNode::isRunning)) {
return false;
}
if (statelessProcessGroups.stream().anyMatch(ProcessGroup::isStatelessActive)) {
return false;
}
return true;
}
@ -707,6 +822,7 @@ public class AffectedComponentSet {
", flowRegistryCliens=" + flowRegistryClients +
", controllerServices=" + controllerServices +
", reportingTasks=" + reportingTasks +
", statelessProcessGroups=" + statelessProcessGroups +
"]";
}
}

View File

@ -21,6 +21,7 @@ import org.apache.nifi.connectable.Port;
import org.apache.nifi.controller.ProcessorNode;
import org.apache.nifi.controller.ReportingTaskNode;
import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.registry.flow.FlowRegistryClientNode;
import org.apache.nifi.remote.RemoteGroupPort;
@ -41,6 +42,9 @@ public interface ComponentSetFilter {
boolean testFlowRegistryClient(FlowRegistryClientNode flowRegistryClient);
boolean testStatelessGroup(ProcessGroup group);
default ComponentSetFilter reverse() {
final ComponentSetFilter original = this;
@ -84,6 +88,11 @@ public interface ComponentSetFilter {
public boolean testFlowRegistryClient(final FlowRegistryClientNode flowRegistryClient) {
return !original.testFlowRegistryClient(flowRegistryClient);
}
@Override
public boolean testStatelessGroup(final ProcessGroup group) {
return !original.testStatelessGroup(group);
}
};
}
}

View File

@ -22,6 +22,7 @@ import org.apache.nifi.controller.ProcessorNode;
import org.apache.nifi.controller.ReportingTaskNode;
import org.apache.nifi.controller.flow.VersionedDataflow;
import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.flow.ExecutionEngine;
import org.apache.nifi.flow.ScheduledState;
import org.apache.nifi.flow.VersionedControllerService;
import org.apache.nifi.flow.VersionedPort;
@ -30,6 +31,7 @@ import org.apache.nifi.flow.VersionedProcessor;
import org.apache.nifi.flow.VersionedRemoteGroupPort;
import org.apache.nifi.flow.VersionedRemoteProcessGroup;
import org.apache.nifi.flow.VersionedReportingTask;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.registry.flow.FlowRegistryClientNode;
import org.apache.nifi.remote.RemoteGroupPort;
@ -43,6 +45,7 @@ public class RunningComponentSetFilter implements ComponentSetFilter {
private final Map<String, VersionedPort> outputPorts = new HashMap<>();
private final Map<String, VersionedRemoteGroupPort> remoteInputPorts = new HashMap<>();
private final Map<String, VersionedRemoteGroupPort> remoteOutputPorts = new HashMap<>();
private final Map<String, VersionedProcessGroup> statelessGroups = new HashMap<>();
public RunningComponentSetFilter(final VersionedDataflow dataflow) {
dataflow.getControllerServices().forEach(service -> controllerServices.put(service.getInstanceIdentifier(), service));
@ -55,6 +58,9 @@ public class RunningComponentSetFilter implements ComponentSetFilter {
group.getOutputPorts().forEach(port -> outputPorts.put(port.getInstanceIdentifier(), port));
group.getControllerServices().forEach(service -> controllerServices.put(service.getInstanceIdentifier(), service));
group.getProcessors().forEach(processor -> processors.put(processor.getInstanceIdentifier(), processor));
if (group.getExecutionEngine() == ExecutionEngine.STATELESS) {
statelessGroups.put(group.getIdentifier(), group);
}
for (final VersionedRemoteProcessGroup rpg : group.getRemoteProcessGroups()) {
rpg.getInputPorts().forEach(port -> {
@ -119,4 +125,10 @@ public class RunningComponentSetFilter implements ComponentSetFilter {
public boolean testFlowRegistryClient(final FlowRegistryClientNode flowRegistryClient) {
return false;
}
@Override
public boolean testStatelessGroup(final ProcessGroup group) {
final VersionedProcessGroup versionedGroup = statelessGroups.get(group.getIdentifier());
return versionedGroup != null && versionedGroup.getScheduledState() == ScheduledState.RUNNING;
}
}

View File

@ -20,6 +20,8 @@ package org.apache.nifi.controller.serialization;
import org.apache.nifi.connectable.Port;
import org.apache.nifi.controller.ProcessorNode;
import org.apache.nifi.controller.ScheduledState;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.groups.StatelessGroupScheduledState;
public interface ScheduledStateLookup {
@ -27,6 +29,8 @@ public interface ScheduledStateLookup {
ScheduledState getScheduledState(Port port);
ScheduledState getScheduledState(ProcessGroup processGroup);
ScheduledStateLookup IDENTITY_LOOKUP = new ScheduledStateLookup() {
@Override
@ -38,5 +42,10 @@ public interface ScheduledStateLookup {
public ScheduledState getScheduledState(final Port port) {
return port.getScheduledState();
}
@Override
public ScheduledState getScheduledState(final ProcessGroup processGroup) {
return processGroup.getDesiredStatelessScheduledState() == StatelessGroupScheduledState.RUNNING ? ScheduledState.RUNNING : ScheduledState.STOPPED;
}
};
}

View File

@ -25,11 +25,11 @@ import org.apache.nifi.controller.ReportingTaskNode;
import org.apache.nifi.controller.Template;
import org.apache.nifi.controller.flow.VersionedDataflow;
import org.apache.nifi.controller.flow.VersionedFlowEncodingVersion;
import org.apache.nifi.flow.VersionedFlowRegistryClient;
import org.apache.nifi.controller.flow.VersionedTemplate;
import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.flow.ScheduledState;
import org.apache.nifi.flow.VersionedControllerService;
import org.apache.nifi.flow.VersionedFlowRegistryClient;
import org.apache.nifi.flow.VersionedParameterContext;
import org.apache.nifi.flow.VersionedParameterProvider;
import org.apache.nifi.flow.VersionedProcessGroup;
@ -217,6 +217,11 @@ public class VersionedDataflowMapper {
return ScheduledState.DISABLED;
}
}
@Override
public ScheduledState getState(final ProcessGroup group) {
return map(stateLookup.getScheduledState(group));
}
};
}
}

View File

@ -40,7 +40,6 @@ import org.apache.nifi.controller.UninheritableFlowException;
import org.apache.nifi.controller.flow.FlowManager;
import org.apache.nifi.controller.flow.VersionedDataflow;
import org.apache.nifi.controller.flow.VersionedFlowEncodingVersion;
import org.apache.nifi.flow.VersionedFlowRegistryClient;
import org.apache.nifi.controller.flow.VersionedTemplate;
import org.apache.nifi.controller.inheritance.AuthorizerCheck;
import org.apache.nifi.controller.inheritance.BundleCompatibilityCheck;
@ -52,11 +51,13 @@ import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.encrypt.EncryptionException;
import org.apache.nifi.encrypt.PropertyEncryptor;
import org.apache.nifi.flow.Bundle;
import org.apache.nifi.flow.ExecutionEngine;
import org.apache.nifi.flow.ScheduledState;
import org.apache.nifi.flow.VersionedComponent;
import org.apache.nifi.flow.VersionedConfigurableExtension;
import org.apache.nifi.flow.VersionedControllerService;
import org.apache.nifi.flow.VersionedExternalFlow;
import org.apache.nifi.flow.VersionedFlowRegistryClient;
import org.apache.nifi.flow.VersionedParameter;
import org.apache.nifi.flow.VersionedParameterContext;
import org.apache.nifi.flow.VersionedParameterProvider;
@ -109,7 +110,6 @@ import java.io.IOException;
import java.io.InputStream;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
@ -451,16 +451,19 @@ public class VersionedFlowSynchronizer implements FlowSynchronizer {
private FlowComparison compareFlows(final DataFlow existingFlow, final DataFlow proposedFlow, final PropertyEncryptor encryptor) {
final DifferenceDescriptor differenceDescriptor = new StaticDifferenceDescriptor();
final VersionedDataflow existingVersionedFlow = existingFlow.getVersionedDataflow() == null ? createEmptyVersionedDataflow() : existingFlow.getVersionedDataflow();
final VersionedDataflow clusterVersionedFlow = proposedFlow.getVersionedDataflow();
final ComparableDataFlow clusterDataFlow = new StandardComparableDataFlow(
"Cluster Flow", clusterVersionedFlow.getRootGroup(), toSet(clusterVersionedFlow.getControllerServices()), toSet(clusterVersionedFlow.getReportingTasks()),
toSet(clusterVersionedFlow.getParameterContexts()), toSet(clusterVersionedFlow.getParameterProviders()), toSet(clusterVersionedFlow.getRegistries()));
final VersionedProcessGroup proposedRootGroup = clusterVersionedFlow.getRootGroup();
final String proposedRootGroupId = proposedRootGroup == null ? null : proposedRootGroup.getInstanceIdentifier();
final VersionedDataflow existingVersionedFlow = existingFlow.getVersionedDataflow() == null ? createEmptyVersionedDataflow(proposedRootGroupId) : existingFlow.getVersionedDataflow();
final ComparableDataFlow localDataFlow = new StandardComparableDataFlow(
"Local Flow", existingVersionedFlow.getRootGroup(), toSet(existingVersionedFlow.getControllerServices()), toSet(existingVersionedFlow.getReportingTasks()),
toSet(existingVersionedFlow.getParameterContexts()),toSet(existingVersionedFlow.getParameterProviders()), toSet(existingVersionedFlow.getRegistries()));
final VersionedDataflow clusterVersionedFlow = proposedFlow.getVersionedDataflow();
final ComparableDataFlow clusterDataFlow = new StandardComparableDataFlow(
"Cluster Flow", clusterVersionedFlow.getRootGroup(), toSet(clusterVersionedFlow.getControllerServices()), toSet(clusterVersionedFlow.getReportingTasks()),
toSet(clusterVersionedFlow.getParameterContexts()), toSet(clusterVersionedFlow.getParameterProviders()), toSet(clusterVersionedFlow.getRegistries()));
final FlowComparator flowComparator = new StandardFlowComparator(localDataFlow, clusterDataFlow, Collections.emptySet(),
differenceDescriptor, encryptor::decrypt, VersionedComponent::getInstanceIdentifier, FlowComparatorVersionedStrategy.DEEP);
return flowComparator.compare();
@ -474,7 +477,7 @@ public class VersionedFlowSynchronizer implements FlowSynchronizer {
return new HashSet<>(values);
}
private VersionedDataflow createEmptyVersionedDataflow() {
private VersionedDataflow createEmptyVersionedDataflow(final String rootGroupId) {
final VersionedDataflow dataflow = new VersionedDataflow();
dataflow.setControllerServices(Collections.emptyList());
dataflow.setEncodingVersion(new VersionedFlowEncodingVersion(2, 0));
@ -482,7 +485,11 @@ public class VersionedFlowSynchronizer implements FlowSynchronizer {
dataflow.setParameterProviders(Collections.emptyList());
dataflow.setRegistries(Collections.emptyList());
dataflow.setReportingTasks(Collections.emptyList());
dataflow.setRootGroup(new VersionedProcessGroup());
final VersionedProcessGroup rootGroup = new VersionedProcessGroup();
rootGroup.setInstanceIdentifier(rootGroupId);
dataflow.setRootGroup(rootGroup);
return dataflow;
}
@ -1239,6 +1246,11 @@ public class VersionedFlowSynchronizer implements FlowSynchronizer {
@Override
public void startNow(final Connectable component) {
if (ExecutionEngine.STATELESS == component.getProcessGroup().resolveExecutionEngine()) {
logger.info("{} should be running but will not start it because its Process Group is configured to run Stateless", component);
return;
}
switch (component.getConnectableType()) {
case PROCESSOR:
flowController.startProcessor(component.getProcessGroupIdentifier(), component.getIdentifier());
@ -1259,13 +1271,13 @@ public class VersionedFlowSynchronizer implements FlowSynchronizer {
flowController.stopConnectable(component);
}
@Override
protected void enableNow(final Collection<ControllerServiceNode> controllerServices) {
flowController.getControllerServiceProvider().enableControllerServices(controllerServices);
}
protected void startNow(final ReportingTaskNode reportingTask) {
flowController.startReportingTask(reportingTask);
}
@Override
protected void startNow(final ProcessGroup statelessGroup) {
flowController.startProcessGroup(statelessGroup);
}
}
}

View File

@ -69,7 +69,7 @@ public class ConnectableTask {
private final SchedulingAgent schedulingAgent;
private final Connectable connectable;
private final RepositoryContext repositoryContext;
private final LifecycleState scheduleState;
private final LifecycleState lifecycleState;
private final ProcessContext processContext;
private final FlowController flowController;
private final int numRelationships;
@ -79,19 +79,19 @@ public class ConnectableTask {
private final int perfTrackingNthIteration;
public ConnectableTask(final SchedulingAgent schedulingAgent, final Connectable connectable,
final FlowController flowController, final RepositoryContextFactory contextFactory, final LifecycleState scheduleState) {
final FlowController flowController, final RepositoryContextFactory contextFactory, final LifecycleState lifecycleState) {
this.schedulingAgent = schedulingAgent;
this.connectable = connectable;
this.scheduleState = scheduleState;
this.lifecycleState = lifecycleState;
this.numRelationships = connectable.getRelationships().size();
this.flowController = flowController;
this.threadMXBean = ManagementFactory.getThreadMXBean();
final StateManager stateManager = new TaskTerminationAwareStateManager(flowController.getStateManagerProvider().getStateManager(connectable.getIdentifier()), scheduleState::isTerminated);
final StateManager stateManager = new TaskTerminationAwareStateManager(flowController.getStateManagerProvider().getStateManager(connectable.getIdentifier()), lifecycleState::isTerminated);
if (connectable instanceof ProcessorNode) {
processContext = new StandardProcessContext(
(ProcessorNode) connectable, flowController.getControllerServiceProvider(), stateManager, scheduleState::isTerminated, flowController);
(ProcessorNode) connectable, flowController.getControllerServiceProvider(), stateManager, lifecycleState::isTerminated, flowController);
} else {
processContext = new ConnectableProcessContext(connectable, stateManager);
}
@ -165,7 +165,7 @@ public class ConnectableTask {
}
public InvocationResult invoke() {
if (scheduleState.isTerminated()) {
if (lifecycleState.isTerminated()) {
logger.debug("Will not trigger {} because task is terminated", connectable);
return InvocationResult.DO_NOT_YIELD;
}
@ -218,17 +218,17 @@ public class ConnectableTask {
final StandardProcessSession rawSession;
final boolean batch;
if (connectable.isSessionBatchingSupported() && batchNanos > 0L) {
rawSession = new StandardProcessSession(repositoryContext, scheduleState::isTerminated, performanceTracker);
rawSession = new StandardProcessSession(repositoryContext, lifecycleState::isTerminated, performanceTracker);
sessionFactory = new BatchingSessionFactory(rawSession);
batch = true;
} else {
rawSession = null;
sessionFactory = new StandardProcessSessionFactory(repositoryContext, scheduleState::isTerminated, performanceTracker);
sessionFactory = new StandardProcessSessionFactory(repositoryContext, lifecycleState::isTerminated, performanceTracker);
batch = false;
}
final ActiveProcessSessionFactory activeSessionFactory = new WeakHashMapProcessSessionFactory(sessionFactory);
scheduleState.incrementActiveThreadCount(activeSessionFactory);
lifecycleState.incrementActiveThreadCount(activeSessionFactory);
final long startNanos = System.nanoTime();
final long finishIfBackpressureEngaged = startNanos + (batchNanos / 25L);
@ -305,7 +305,7 @@ public class ConnectableTask {
logger.error("", e);
}
} finally {
scheduleState.decrementActiveThreadCount();
lifecycleState.decrementActiveThreadCount();
Thread.currentThread().setName(originalThreadName);
}
}

View File

@ -0,0 +1,899 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller.tasks;
import org.apache.nifi.connectable.Connection;
import org.apache.nifi.connectable.ConnectionUtils;
import org.apache.nifi.connectable.ConnectionUtils.FlowFileCloneResult;
import org.apache.nifi.connectable.Port;
import org.apache.nifi.controller.ProcessorNode;
import org.apache.nifi.controller.ScheduledState;
import org.apache.nifi.controller.Triggerable;
import org.apache.nifi.controller.queue.FlowFileQueue;
import org.apache.nifi.controller.repository.ContentRepository;
import org.apache.nifi.controller.repository.FlowFileEvent;
import org.apache.nifi.controller.repository.FlowFileEventRepository;
import org.apache.nifi.controller.repository.FlowFileRecord;
import org.apache.nifi.controller.repository.FlowFileRepository;
import org.apache.nifi.controller.repository.RepositoryRecord;
import org.apache.nifi.controller.repository.RepositoryRecordType;
import org.apache.nifi.controller.repository.StandardFlowFileRecord;
import org.apache.nifi.controller.repository.StandardRepositoryRecord;
import org.apache.nifi.controller.repository.metrics.StandardFlowFileEvent;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.groups.StatelessGroupNode;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.exception.TerminatedTaskException;
import org.apache.nifi.provenance.ProvenanceEventBuilder;
import org.apache.nifi.provenance.ProvenanceEventRecord;
import org.apache.nifi.provenance.ProvenanceEventRepository;
import org.apache.nifi.provenance.ProvenanceEventType;
import org.apache.nifi.provenance.StandardProvenanceEventRecord;
import org.apache.nifi.scheduling.SchedulingStrategy;
import org.apache.nifi.stateless.flow.DataflowTrigger;
import org.apache.nifi.stateless.flow.DataflowTriggerContext;
import org.apache.nifi.stateless.flow.FailurePortEncounteredException;
import org.apache.nifi.stateless.flow.FlowFileSupplier;
import org.apache.nifi.stateless.flow.StatelessDataflow;
import org.apache.nifi.stateless.flow.TriggerResult;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.function.Predicate;
import java.util.stream.Collectors;
public class StatelessFlowTask {
private static final Set<ProvenanceEventType> eventTypesToKeepOnFailure = EnumSet.of(ProvenanceEventType.SEND, ProvenanceEventType.REMOTE_INVOCATION);
private final StatelessGroupNode statelessGroupNode;
private final StatelessDataflow flow;
private final FlowFileRepository nifiFlowFileRepository;
private final ProvenanceEventRepository nifiProvenanceEventRepository;
private final ContentRepository nifiContentRepository;
private final FlowFileEventRepository flowFileEventRepository;
private final ComponentLog logger;
private final Map<String, Port> outputPorts;
private final long timeoutMillis;
private final boolean allowBatch;
// State that is updated during invocation - these variables are guarded by synchronized block
private Long maxProvenanceEventId;
private List<FlowFileCloneResult> cloneResults;
private List<RepositoryRecord> outputRepositoryRecords;
private List<ProvenanceEventRecord> cloneProvenanceEvents;
private StatelessFlowTask(final Builder builder) {
this.statelessGroupNode = builder.statelessGroupNode;
this.nifiFlowFileRepository = builder.flowFileRepository;
this.nifiContentRepository = builder.contentRepository;
this.nifiProvenanceEventRepository = builder.provenanceEventRepository;
this.flowFileEventRepository = builder.flowFileEventRepository;
this.flow = builder.statelessFlow;
this.allowBatch = isAllowBatch(statelessGroupNode.getProcessGroup());
this.logger = builder.logger;
this.timeoutMillis = builder.timeoutMillis;
final ProcessGroup processGroup = statelessGroupNode.getProcessGroup();
outputPorts = new HashMap<>();
for (final Port outputPort : processGroup.getOutputPorts()) {
outputPorts.put(outputPort.getName(), outputPort);
}
}
private boolean isAllowBatch(final ProcessGroup group) {
// We allow batch only if there are no processors that use @TriggerSerially.
// If any Processor does require being triggered serially, it may indicate that it is not allowed
// to perform multiple invocations without first committing its session. In this case, we cannot
// use multiple threads, and we cannot use batch processing.
return group.findAllProcessors().stream()
.noneMatch(this::isPreventBatch);
}
private boolean isPreventBatch(final ProcessorNode procNode) {
if (procNode.isTriggeredSerially()) {
return true;
}
if (procNode.hasIncomingConnection()) {
return false;
}
if (!isRunAsFastAsPossible(procNode)) {
return true;
}
return false;
}
private boolean isRunAsFastAsPossible(final ProcessorNode procNode) {
final SchedulingStrategy schedulingStrategy = procNode.getSchedulingStrategy();
if (schedulingStrategy != SchedulingStrategy.TIMER_DRIVEN) {
return false;
}
if (procNode.getSchedulingPeriod(TimeUnit.NANOSECONDS) > Triggerable.MINIMUM_SCHEDULING_NANOS) {
return false;
}
return true;
}
public void shutdown() {
this.flow.shutdown(false, true);
}
private boolean isAbort() {
final ScheduledState desiredState = statelessGroupNode.getDesiredState();
return desiredState != ScheduledState.RUNNING && desiredState != ScheduledState.RUN_ONCE;
}
public synchronized void trigger() {
final long startTime = System.currentTimeMillis();
final long endTime = startTime + 100L;
if (allowBatch) {
logger.debug("Will run in batch mode for 100 milliseconds until {}", endTime);
}
final List<Invocation> allInvocations = new ArrayList<>();
final List<Invocation> successfulInvocations = new ArrayList<>();
final ProvenanceEventRepository statelessProvRepo = flow.getProvenanceRepository();
maxProvenanceEventId = statelessProvRepo.getMaxEventId();
try {
int invocationCount = 0;
while ((invocationCount == 0 || allowBatch) && System.currentTimeMillis() < endTime) {
invocationCount++;
final Invocation invocation = new Invocation();
final FlowFileSupplier flowFileSupplier = new BridgingFlowFileSupplier(invocation);
final DataflowTriggerContext triggerContext = new StatelessFlowTaskTriggerContext(flowFileSupplier);
final TriggerResult triggerResult = triggerFlow(triggerContext);
invocation.setTriggerResult(triggerResult);
allInvocations.add(invocation);
if (triggerResult.isSuccessful()) {
successfulInvocations.add(invocation);
// If we pulled in more than 1 FlowFile for this invocation, do not trigger again.
// We do this because most of the time when multiple FlowFiles are pulled in, it's for a merge, etc.
// and in that case, there may well not be enough FlowFiles for another invocation to succeed. We don't
// want to hold up this invocation until the flow times out, so we just stop triggering.
if (invocation.getPolledFlowFiles().size() > 1) {
break;
}
} else {
logger.debug("Failed to trigger", triggerResult.getFailureCause().orElse(null));
fail(invocation);
break;
}
}
logger.debug("Finished triggering");
} finally {
try {
completeInvocations(successfulInvocations);
} catch (final Exception e) {
logger.error("Failed to complete Stateless Flow", e);
statelessGroupNode.yield();
fail(successfulInvocations, e);
}
logger.debug("Acknowledging FlowFiles from {} invocations", allInvocations.size());
for (final Invocation invocation : allInvocations) {
for (final PolledFlowFile polledFlowFile : invocation.getPolledFlowFiles()) {
polledFlowFile.getOriginalQueue().acknowledge(polledFlowFile.getInputFlowFile());
}
}
}
}
private void fail(final List<Invocation> invocations, final Throwable cause) {
invocations.forEach(invocation -> fail(invocation, cause));
}
private void fail(final Invocation invocation) {
final Throwable cause;
if (invocation.getTriggerResult().isCanceled()) {
cause = new TerminatedTaskException();
} else {
cause = invocation.getTriggerResult().getFailureCause().orElse(null);
}
fail(invocation, cause);
}
private void fail(final Invocation invocation, final Throwable cause) {
final Port destinationPort = getDestinationPort(cause);
try {
failInvocation(invocation, destinationPort, cause);
} catch (final Exception e) {
if (cause != null) {
cause.addSuppressed(e);
}
logger.error("Failed to trigger Stateless Flow and failed to properly handle failure", cause);
}
}
private Port getDestinationPort(final Throwable failureCause) {
if (!(failureCause instanceof FailurePortEncounteredException)) {
return null;
}
final FailurePortEncounteredException fpee = (FailurePortEncounteredException) failureCause;
final Port port = this.outputPorts.get(fpee.getPortName());
if (port == null) {
logger.error("FlowFile was routed to Failure Port {} but no such port exists in the dataflow", fpee.getPortName());
}
return port;
}
private TriggerResult triggerFlow(final DataflowTriggerContext triggerContext) {
final DataflowTrigger trigger = flow.trigger(triggerContext);
try {
final Optional<TriggerResult> optionalResult = trigger.getResult(timeoutMillis, TimeUnit.MILLISECONDS);
if (optionalResult.isEmpty()) {
trigger.cancel();
return trigger.getResult(5, TimeUnit.SECONDS)
.orElseThrow(() -> new ProcessException("Stateless Flow " + this + " timed out and failed to cancel within the allotted amount of time"));
}
return optionalResult.get();
} catch (final InterruptedException e) {
Thread.currentThread().interrupt();
throw new RuntimeException(e);
}
}
private void completeInvocations(final List<Invocation> invocations) throws IOException {
logger.debug("Completing transactions from {} invocations", invocations.size());
if (invocations.isEmpty()) {
return;
}
resetState();
// Validate that the outputs are valid before we do anything else.
for (final Invocation invocation : invocations) {
validateDestinations(invocation.getTriggerResult());
}
// Drop the input FlowFiles from the FlowFile Repository. This will also ensure that the Claimant Count
// gets updated for the input FlowFiles and that the claims are properly destroyed if necessary.
for (final Invocation invocation : invocations) {
dropInputFlowFiles(invocation);
}
// Determine where the FlowFiles are going and create the appropriate RepositoryRecords, populating
// this.outputRepositoryRecords and this.cloneResults
for (final Invocation invocation : invocations) {
createOutputRecords(invocation.getTriggerResult().getOutputFlowFiles());
}
// Update the claimant counts based on output FlowFiles.
updateClaimantCounts();
// Update the FlowFile Repository to reflect that the input FlowFiles were dropped and the output FlowFiles were created.
try {
updateFlowFileRepository();
} catch (final Exception e) {
throw new IOException("Failed to update FlowFile Repository after triggering " + this, e);
}
updateProvenanceRepository(event -> true);
// Acknowledge the invocations so that the sessions can be committed
for (final Invocation invocation : invocations) {
acknowledge(invocation);
}
updateEventRepository(invocations);
// Distribute FlowFiles to the outbound queues
distributeFlowFiles();
}
void resetState() {
cloneResults = new ArrayList<>();
outputRepositoryRecords = new ArrayList<>();
cloneProvenanceEvents = new ArrayList<>();
}
private void failInvocation(final Invocation invocation, final Port destinationPort, final Throwable cause) throws IOException {
final List<PolledFlowFile> inputFlowFiles = invocation.getPolledFlowFiles();
boolean stopped = false;
if (cause instanceof TerminatedTaskException) {
final String input = inputFlowFiles.isEmpty() ? "no input FlowFile" : inputFlowFiles.toString();
// A TerminatedTaskException will happen for 2 reasons: the group was stopped, or it timed out.
// If it was stopped, just log at an INFO level, as this is expected. If it timed out, log an error.
final ScheduledState desiredState = this.statelessGroupNode.getDesiredState();
if (desiredState == ScheduledState.STOPPED) {
logger.info("Stateless Flow canceled while running with input {}", input);
stopped = true;
} else {
logger.error("Stateless Flow timed out while running with input {}", input);
}
} else {
final String input = inputFlowFiles.isEmpty() ? "with no input FlowFile" : " for input " + inputFlowFiles;
logger.error("Failed to trigger Stateless Flow {}", input, cause);
}
resetState();
// While the successful path calls dropInputFlowFile, we do not do this in the failure path because on failure
// the Stateless Flow will be purged, and this will handle dropping all FlowFiles that are in the flow.
// Determine where the FlowFiles are going and create the appropriate RepositoryRecords, populating
// this.outputRepositoryRecords and this.cloneResults
if (!inputFlowFiles.isEmpty()) {
if (destinationPort == null) {
// There is no destination port. Create a penalized FlowFile and place it back on its original queue.
// Do not penalize FlowFiles if the group was stopped, though. That is not a true failure.
for (final PolledFlowFile polledFlowFile : inputFlowFiles) {
if (stopped) {
polledFlowFile.getOriginalQueue().put(polledFlowFile.getInputFlowFile());
} else {
final FlowFileRecord newFile = new StandardFlowFileRecord.Builder()
.fromFlowFile(polledFlowFile.getInputFlowFile())
.penaltyExpirationTime(System.currentTimeMillis() + 30_000L)
.build();
polledFlowFile.getOriginalQueue().put(newFile);
}
}
} else {
dropInputFlowFiles(invocation);
final Map<String, List<FlowFile>> outputRecords = Collections.singletonMap(destinationPort.getName(),
inputFlowFiles.stream().map(PolledFlowFile::getInputFlowFile).collect(Collectors.toList()));
createOutputRecords(outputRecords);
// Update the claimant counts based on output FlowFiles.
updateClaimantCounts();
}
}
// Update the FlowFile Repository to reflect that the input FlowFiles were dropped and the output FlowFiles were created.
try {
updateFlowFileRepository();
} catch (final Exception e) {
throw new IOException("Failed to update FlowFile Repository after triggering " + this, e);
}
updateProvenanceRepository(event -> eventTypesToKeepOnFailure.contains(event.getEventType()));
// Acknowledge the invocations so that the sessions can be committed
abort(invocation, cause);
updateEventRepository(Collections.singletonList(invocation));
// Distribute FlowFiles to the outbound queues
distributeFlowFiles();
}
private void validateDestinations(final TriggerResult result) {
final Map<String, List<FlowFile>> outputFlowFiles = result.getOutputFlowFiles();
// Validate the output to ensure that all given port names are valid
for (final Map.Entry<String, List<FlowFile>> entry : outputFlowFiles.entrySet()) {
final String portName = entry.getKey();
final Port outputPort = outputPorts.get(portName);
if (outputPort == null) {
logger.error("Transferred FlowFile to Output Port {} but no port is known with that name", portName);
throw new IllegalStateException("FlowFile was transferred to nonexistent Port " + portName);
}
}
}
// Visible for testing
void dropInputFlowFiles(final Invocation invocation) {
for (final PolledFlowFile polledFlowFile : invocation.getPolledFlowFiles()) {
final FlowFileRecord inputFlowFile = polledFlowFile.getInputFlowFile();
if (inputFlowFile == null) {
continue;
}
final StandardRepositoryRecord repoRecord = new StandardRepositoryRecord(polledFlowFile.getOriginalQueue(), polledFlowFile.getInputFlowFile());
repoRecord.markForDelete();
outputRepositoryRecords.add(repoRecord);
}
}
List<RepositoryRecord> getOutputRepositoryRecords() {
return outputRepositoryRecords;
}
public List<FlowFileCloneResult> getCloneResults() {
return cloneResults;
}
public List<ProvenanceEventRecord> getCloneProvenanceEvents() {
return cloneProvenanceEvents;
}
void createOutputRecords(final Map<String, List<FlowFile>> outputFlowFiles) {
for (final Map.Entry<String, List<FlowFile>> entry : outputFlowFiles.entrySet()) {
final String portName = entry.getKey();
final Port outputPort = outputPorts.get(portName);
final List<FlowFileRecord> portFlowFiles = (List) entry.getValue();
final Set<Connection> outputConnections = outputPort.getConnections();
for (final FlowFileRecord outputFlowFile : portFlowFiles) {
final FlowFileCloneResult cloneResult = ConnectionUtils.clone(outputFlowFile, outputConnections,
nifiFlowFileRepository, null);
cloneResults.add(cloneResult);
final List<RepositoryRecord> repoRecords = cloneResult.getRepositoryRecords();
outputRepositoryRecords.addAll(repoRecords);
// If we generated any clones, create provenance events for them.
createCloneProvenanceEvent(outputFlowFile, repoRecords, outputPort).ifPresent(cloneProvenanceEvents::add);
}
}
}
void updateProvenanceRepository(final Predicate<ProvenanceEventRecord> eventFilter) {
long firstProvEventId = (maxProvenanceEventId == null) ? 0 : (maxProvenanceEventId + 1);
final ProvenanceEventRepository statelessProvRepo = flow.getProvenanceRepository();
if (!cloneProvenanceEvents.isEmpty()){
nifiProvenanceEventRepository.registerEvents(cloneProvenanceEvents);
}
while (true) {
try {
final List<ProvenanceEventRecord> statelessProvEvents = statelessProvRepo.getEvents(firstProvEventId, 1000);
if (statelessProvEvents.isEmpty()) {
return;
}
// We don't want to use the ProvenanceEventRecord objects directly because they already have their IDs populated.
// Instead, we will create a new Provenance Event from the Stateless Event, but the #fromEvent method does not
// copy the Event ID.
final List<ProvenanceEventRecord> provenanceEvents = new ArrayList<>();
for (final ProvenanceEventRecord eventRecord : statelessProvEvents) {
if (eventFilter.test(eventRecord) == false) {
continue;
}
provenanceEvents.add(new StandardProvenanceEventRecord.Builder()
.fromEvent(eventRecord)
.build());
}
nifiProvenanceEventRepository.registerEvents(provenanceEvents);
if (provenanceEvents.size() == 1000) {
firstProvEventId += 1000;
} else {
break;
}
} catch (final IOException e) {
logger.warn("Failed to obtain Provenance Events from Stateless Dataflow. These events will not be added to the NiFi Provenance Repository", e);
}
}
}
void updateClaimantCounts() {
// Update Claimant Counts. The Stateless engine will decrement the Claimant Count for each output FlowFile, as it no longer is responsible for
// the FlowFiles and their content. Now, we are taking ownership of them and need to increment the Claimant Count for each FlowFile.
// But we also no longer have the input FlowFile in its queue. So we need to decrement its claimant count. We could try to be more clever and
// only decrement the claimant count if the input FlowFile was not transferred out, but simply incrementing for each output FlowFiles and decrementing
// for the input FlowFile is simpler and will work just as well.
for (final RepositoryRecord outputRepoRecord : outputRepositoryRecords) {
if (outputRepoRecord.getType() != RepositoryRecordType.DELETE) {
nifiContentRepository.incrementClaimaintCount(outputRepoRecord.getCurrentClaim());
}
}
}
private void updateFlowFileRepository() throws IOException {
// Update the FlowFile repository
nifiFlowFileRepository.updateRepository(outputRepositoryRecords);
}
private void acknowledge(final Invocation invocation) {
invocation.getTriggerResult().acknowledge();
}
private void abort(final Invocation invocation, final Throwable cause) {
invocation.getTriggerResult().abort(cause);
}
private void distributeFlowFiles() {
int enqueued = 0;
for (final FlowFileCloneResult result : cloneResults) {
enqueued += result.distributeFlowFiles();
}
logger.debug("Distributed {} FlowFiles to output queues", enqueued);
}
void updateEventRepository(final List<Invocation> invocations) {
final Map<String, StandardFlowFileEvent> eventsByComponentId = new HashMap<>();
for (final Invocation invocation : invocations) {
final List<PolledFlowFile> polledFlowFiles = invocation.getPolledFlowFiles();
for (final PolledFlowFile polledFlowFile : polledFlowFiles) {
final long bytes = polledFlowFile.getInputFlowFile().getSize();
final int numOutputConnections = polledFlowFile.getInputPort().getConnections().size();
final StandardFlowFileEvent flowFileEvent = new StandardFlowFileEvent();
flowFileEvent.setFlowFilesIn(1);
flowFileEvent.setFlowFilesOut(numOutputConnections);
flowFileEvent.setContentSizeIn(bytes);
flowFileEvent.setContentSizeOut(numOutputConnections * bytes);
final StandardFlowFileEvent cumulativeEvent = eventsByComponentId.computeIfAbsent(polledFlowFile.getInputPort().getIdentifier(), key -> new StandardFlowFileEvent());
cumulativeEvent.add(flowFileEvent);
}
final Map<String, List<FlowFile>> outputFlowFiles = invocation.getTriggerResult().getOutputFlowFiles();
for (final Map.Entry<String, List<FlowFile>> entry : outputFlowFiles.entrySet()) {
final String portName = entry.getKey();
final List<FlowFile> flowFiles = entry.getValue();
int flowFileCount = flowFiles.size();
long byteCount = 0L;
for (final FlowFile flowFile : flowFiles) {
byteCount += flowFile.getSize();
}
final Port port = outputPorts.get(portName);
final int outputConnectionCount = port.getConnections().size();
final StandardFlowFileEvent flowFileEvent = new StandardFlowFileEvent();
flowFileEvent.setFlowFilesIn(flowFileCount);
flowFileEvent.setFlowFilesOut(flowFileCount * outputConnectionCount);
flowFileEvent.setContentSizeIn(byteCount);
flowFileEvent.setContentSizeOut(byteCount * outputConnectionCount);
final StandardFlowFileEvent cumulativeEvent = eventsByComponentId.computeIfAbsent(port.getIdentifier(), key -> new StandardFlowFileEvent());
cumulativeEvent.add(flowFileEvent);
}
}
for (final Map.Entry<String, StandardFlowFileEvent> entry : eventsByComponentId.entrySet()) {
final String componentId = entry.getKey();
final FlowFileEvent event = entry.getValue();
try {
flowFileEventRepository.updateRepository(event, componentId);
} catch (final Exception e) {
logger.warn("Failed to update FlowFile Event Repository", e);
}
}
}
private void expireRecords(final FlowFileQueue sourceQueue, final Set<FlowFileRecord> expiredRecords) throws IOException {
if (expiredRecords.isEmpty()) {
return;
}
final List<RepositoryRecord> repositoryRecords = new ArrayList<>();
final long time = System.currentTimeMillis();
final String expirationDetails = "Expiration Threshold = " + sourceQueue.getFlowFileExpiration();
for (final FlowFileRecord expired : expiredRecords) {
final StandardRepositoryRecord record = new StandardRepositoryRecord(sourceQueue, expired);
record.markForDelete();
repositoryRecords.add(record);
}
// Create an Iterable<ProvenanceEventRecord> instead of creating a List<ProvenanceEventRecord> in order to avoid
// the heap consumption, since this can correlate to a lot of Provenance Events.
final Iterable<ProvenanceEventRecord> provenanceEventIterable = new Iterable<>() {
@Override
public Iterator<ProvenanceEventRecord> iterator() {
final Iterator<FlowFileRecord> expiredItr = expiredRecords.iterator();
return new Iterator<>() {
@Override
public boolean hasNext() {
return expiredItr.hasNext();
}
@Override
public ProvenanceEventRecord next() {
final FlowFileRecord expired = expiredItr.next();
final ProvenanceEventRecord provenanceEvent = new StandardProvenanceEventRecord.Builder()
.fromFlowFile(expired)
.setEventTime(time)
.setEventType(ProvenanceEventType.EXPIRE)
.setDetails(expirationDetails)
.setComponentId(sourceQueue.getIdentifier())
.setComponentType("Connection")
.build();
return provenanceEvent;
}
};
}
};
nifiFlowFileRepository.updateRepository(repositoryRecords);
nifiProvenanceEventRepository.registerEvents(provenanceEventIterable);
expiredRecords.clear();
}
private Optional<ProvenanceEventRecord> createCloneProvenanceEvent(final FlowFileRecord outputFlowFile, final List<RepositoryRecord> cloneRecords, final Port outputPort) {
if (outputFlowFile == null || cloneRecords == null || cloneRecords.size() < 2) {
return Optional.empty();
}
final ProvenanceEventBuilder builder = new StandardProvenanceEventRecord.Builder()
.setEventType(ProvenanceEventType.CLONE)
.fromFlowFile(outputFlowFile)
.setLineageStartDate(outputFlowFile.getLineageStartDate())
.setComponentId(outputPort.getIdentifier())
.setComponentType("Output Port")
.addParentFlowFile(outputFlowFile);
for (final RepositoryRecord clone : cloneRecords) {
final FlowFileRecord childFlowFile = clone.getCurrent();
if (outputFlowFile.equals(childFlowFile)) {
continue;
}
builder.addChildFlowFile(childFlowFile);
}
final ProvenanceEventRecord cloneEvent = builder.build();
return Optional.of(cloneEvent);
}
@Override
public String toString() {
return "StatelessFlowTask[Group=" + statelessGroupNode.getProcessGroup() + "]";
}
static class PolledFlowFile {
private final FlowFileRecord inputFlowFile;
private final FlowFileQueue originalQueue;
private final Port inputPort;
public PolledFlowFile(final FlowFileRecord inputFlowFile, final FlowFileQueue originalQueue, final Port inputPort) {
this.inputFlowFile = inputFlowFile;
this.originalQueue = originalQueue;
this.inputPort = inputPort;
}
public FlowFileRecord getInputFlowFile() {
return inputFlowFile;
}
public FlowFileQueue getOriginalQueue() {
return originalQueue;
}
public Port getInputPort() {
return inputPort;
}
@Override
public String toString() {
return inputFlowFile.toString();
}
}
static class Invocation {
private List<PolledFlowFile> polledFlowFiles;
private TriggerResult triggerResult;
public void setTriggerResult(final TriggerResult triggerResult) {
this.triggerResult = triggerResult;
}
public List<PolledFlowFile> getPolledFlowFiles() {
if (polledFlowFiles == null) {
return Collections.emptyList();
}
return polledFlowFiles;
}
public TriggerResult getTriggerResult() {
return triggerResult;
}
public void addPolledFlowFile(final PolledFlowFile polledFlowFile) {
if (polledFlowFile == null) {
return;
}
if (polledFlowFiles == null) {
polledFlowFiles = new ArrayList<>();
}
polledFlowFiles.add(polledFlowFile);
}
}
public static class Builder {
private StatelessGroupNode statelessGroupNode;
private FlowFileRepository flowFileRepository;
private ContentRepository contentRepository;
private ProvenanceEventRepository provenanceEventRepository;
private FlowFileEventRepository flowFileEventRepository;
private StatelessDataflow statelessFlow;
private long timeoutMillis = TimeUnit.MINUTES.toMillis(1);
private ComponentLog logger;
public Builder statelessGroupNode(final StatelessGroupNode statelessGroupNode) {
this.statelessGroupNode = statelessGroupNode;
return this;
}
public Builder nifiFlowFileRepository(final FlowFileRepository flowFileRepository) {
this.flowFileRepository = flowFileRepository;
return this;
}
public Builder nifiContentRepository(final ContentRepository contentRepository) {
this.contentRepository = contentRepository;
return this;
}
public Builder nifiProvenanceRepository(final ProvenanceEventRepository provenanceEventRepository) {
this.provenanceEventRepository = provenanceEventRepository;
return this;
}
public Builder flowFileEventRepository(final FlowFileEventRepository flowFileEventRepository) {
this.flowFileEventRepository = flowFileEventRepository;
return this;
}
public Builder statelessFlow(final StatelessDataflow statelessFlow) {
this.statelessFlow = statelessFlow;
return this;
}
public Builder timeout(final long value, final TimeUnit unit) {
final long millis = unit.toMillis(value);
this.timeoutMillis = Math.max(millis, 1L);
return this;
}
public Builder logger(final ComponentLog logger) {
this.logger = logger;
return this;
}
public StatelessFlowTask build() {
return new StatelessFlowTask(this);
}
}
/**
* A FlowFileSupplier that bridges between the running NiFi instance and the Stateless Engine.
*/
private class BridgingFlowFileSupplier implements FlowFileSupplier {
private final Map<String, Port> portsByName;
private final Set<FlowFileRecord> expiredRecords = new HashSet<>();
private final Invocation invocation;
private int zeroFlowFileInvocations = 0;
public BridgingFlowFileSupplier(final Invocation invocation) {
this.invocation = invocation;
final Set<Port> inputPorts = statelessGroupNode.getProcessGroup().getInputPorts();
portsByName = inputPorts.stream()
.collect(Collectors.toMap(Port::getName, port -> port));
}
@Override
public Optional<FlowFile> getFlowFile(final String portName) {
final Port port = portsByName.get(portName);
if (port == null) {
return Optional.empty();
}
for (final Connection sourceConnection : port.getIncomingConnections()) {
final FlowFileQueue sourceQueue = sourceConnection.getFlowFileQueue();
final FlowFileRecord flowFile = sourceQueue.poll(expiredRecords);
if (!expiredRecords.isEmpty()) {
try {
expireRecords(sourceQueue, expiredRecords);
} catch (final Exception e) {
logger.error("Failed to expire FlowFile Records when consuming from input queue {}", sourceQueue, e);
}
expiredRecords.clear();
}
if (flowFile != null) {
zeroFlowFileInvocations = 0;
// We need to increment the Content Claim for the FlowFile because when we complete the flow, for any FlowFile queued
// up in an Output Port, the Stateless Engine will decrement the claimant count because it no longer owns the FlowFile.
// Incrementing it here ensures that the claimant count is properly counted.
nifiContentRepository.incrementClaimaintCount(flowFile.getContentClaim());
invocation.addPolledFlowFile(new PolledFlowFile(flowFile, sourceQueue, port));
return Optional.of(flowFile);
}
}
// In order to avoid overwhelming the CPU in an unproductive loop, since we have no data we will yield for 10 milliseconds.
// We do not do this for the first invocation, however, as this is only helpful for flows that require multiple FlowFiles.
if (++zeroFlowFileInvocations > 1) {
final long yieldMillis = statelessGroupNode.getBoredYieldDuration(TimeUnit.MILLISECONDS);
try {
Thread.sleep(yieldMillis);
} catch (final InterruptedException ie) {
Thread.currentThread().interrupt();
}
}
return Optional.empty();
}
}
private class StatelessFlowTaskTriggerContext implements DataflowTriggerContext {
private final FlowFileSupplier flowFileSupplier;
public StatelessFlowTaskTriggerContext(final FlowFileSupplier flowFileSupplier) {
this.flowFileSupplier = flowFileSupplier;
}
@Override
public boolean isAbort() {
return StatelessFlowTask.this.isAbort();
}
@Override
public FlowFileSupplier getFlowFileSupplier() {
return flowFileSupplier;
}
}
}

View File

@ -18,6 +18,8 @@ package org.apache.nifi.diagnostics;
import org.apache.nifi.controller.repository.ContentRepository;
import org.apache.nifi.controller.repository.FlowFileRepository;
import org.apache.nifi.controller.repository.claim.ResourceClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
import org.apache.nifi.provenance.ProvenanceRepository;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -32,6 +34,8 @@ import java.lang.management.RuntimeMXBean;
import java.lang.management.ThreadMXBean;
import java.lang.reflect.Method;
import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
@ -45,7 +49,8 @@ public class SystemDiagnosticsFactory {
private final Logger logger = LoggerFactory.getLogger(SystemDiagnosticsFactory.class);
public SystemDiagnostics create(final FlowFileRepository flowFileRepo, final ContentRepository contentRepo, ProvenanceRepository provenanceRepository) {
public SystemDiagnostics create(final FlowFileRepository flowFileRepo, final ContentRepository contentRepo, final ProvenanceRepository provenanceRepository,
final ResourceClaimManager resourceClaimManager) {
final SystemDiagnostics systemDiagnostics = new SystemDiagnostics();
final MemoryMXBean memory = ManagementFactory.getMemoryMXBean();
@ -185,6 +190,27 @@ public class SystemDiagnosticsFactory {
// Ignore. This will throw either ClassNotFound or NoClassDefFoundError if unavailable in this JVM.
}
final Map<ResourceClaim, Integer> claimantCounts = new HashMap<>();
final Set<ResourceClaim> destructableClaims = new HashSet<>();
for (final String containerName : contentRepo.getContainerNames()) {
try {
final Set<ResourceClaim> resourceClaims = contentRepo.getActiveResourceClaims(containerName);
for (final ResourceClaim resourceClaim : resourceClaims) {
final int claimantCount = resourceClaimManager.getClaimantCount(resourceClaim);
claimantCounts.put(resourceClaim, claimantCount);
if (resourceClaimManager.isDestructable(resourceClaim)) {
destructableClaims.add(resourceClaim);
}
}
} catch (final Exception e) {
logger.warn("Failed to determine Resource Claim usage for Content Repository Container {}", containerName, e);
}
}
systemDiagnostics.setClaimantCounts(claimantCounts);
systemDiagnostics.setDestructableClaims(destructableClaims);
// set the creation timestamp
systemDiagnostics.setCreationTimestamp(new Date().getTime());

View File

@ -34,6 +34,7 @@ import org.apache.nifi.controller.kerberos.KerberosConfig;
import org.apache.nifi.controller.repository.FlowFileEventRepository;
import org.apache.nifi.controller.scheduling.LifecycleState;
import org.apache.nifi.controller.scheduling.SchedulingAgent;
import org.apache.nifi.controller.scheduling.StandardLifecycleStateManager;
import org.apache.nifi.controller.scheduling.StandardProcessScheduler;
import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.controller.status.history.StatusHistoryRepository;
@ -156,7 +157,7 @@ public class StandardProcessorNodeIT {
final SchedulingAgentCallback schedulingAgentCallback = new FailIfTriggeredSchedulingAgentCallback(taskScheduler);
procNode.performValidation();
procNode.start(taskScheduler, 20000L, 10000L, () -> processContext, schedulingAgentCallback, true);
procNode.start(taskScheduler, 20000L, 10000L, () -> processContext, schedulingAgentCallback, true, true);
Thread.sleep(1000L);
assertEquals(1, processor.onScheduledCount);
@ -283,7 +284,7 @@ public class StandardProcessorNodeIT {
final StateManagerProvider stateManagerProvider = mock(StateManagerProvider.class);
final ProcessScheduler processScheduler = new StandardProcessScheduler(null, flowController,
stateManagerProvider, nifiProperties);
stateManagerProvider, nifiProperties, new StandardLifecycleStateManager());
final LoggableComponent<Processor> loggableComponent = new LoggableComponent<>(processor, narBundle.getBundleDetails().getCoordinate(), componentLog);
final StandardProcessorNode procNode = new StandardProcessorNode(loggableComponent, uuid, validationContextFactory, processScheduler,
@ -538,14 +539,14 @@ public class StandardProcessorNodeIT {
final StandardProcessContext processContext = new StandardProcessContext(procNode, null, null, () -> false, null);
final SchedulingAgentCallback schedulingAgentCallback = new FailIfTriggeredSchedulingAgentCallback(taskScheduler);
procNode.start(taskScheduler, 20000L, 10000L, () -> processContext, schedulingAgentCallback, true);
procNode.start(taskScheduler, 20000L, 10000L, () -> processContext, schedulingAgentCallback, true, true);
assertEquals(ScheduledState.STARTING, procNode.getPhysicalScheduledState());
final ProcessScheduler processScheduler = mock(ProcessScheduler.class);
final SchedulingAgent schedulingAgent = mock(SchedulingAgent.class);
final LifecycleState lifecycleState = new LifecycleState();
final LifecycleState lifecycleState = new LifecycleState(procNode.getIdentifier());
final Future<Void> future = procNode.stop(processScheduler, taskScheduler, processContext, schedulingAgent, lifecycleState);
final Future<Void> future = procNode.stop(processScheduler, taskScheduler, processContext, schedulingAgent, lifecycleState, true);
final ScheduledState currentState = procNode.getPhysicalScheduledState();
assertTrue(currentState == ScheduledState.STOPPED || currentState == ScheduledState.STOPPING);

View File

@ -646,7 +646,7 @@ public class TestWriteAheadFlowFileRepository {
final FlowFileRecord flowFileRecord = ffBuilder.build();
final List<RepositoryRecord> records = new ArrayList<>();
final StandardRepositoryRecord record = new StandardRepositoryRecord(null);
final StandardRepositoryRecord record = new StandardRepositoryRecord((FlowFileQueue) null);
record.setWorking(flowFileRecord, false);
record.setDestination(connection.getFlowFileQueue());
records.add(record);

View File

@ -19,6 +19,7 @@ package org.apache.nifi.controller.scheduling;
public class DummyScheduleState extends LifecycleState {
public DummyScheduleState(boolean isScheduled) {
super("Dummy Component");
setScheduled(isScheduled);
}

View File

@ -143,7 +143,8 @@ public class TestStandardProcessScheduler {
extensionManager = new StandardExtensionDiscoveringManager();
extensionManager.discoverExtensions(systemBundle, Collections.emptySet());
scheduler = new StandardProcessScheduler(new FlowEngine(1, "Unit Test", true), Mockito.mock(FlowController.class), stateMgrProvider, nifiProperties);
scheduler = new StandardProcessScheduler(new FlowEngine(1, "Unit Test", true), Mockito.mock(FlowController.class),
stateMgrProvider, nifiProperties, new StandardLifecycleStateManager());
scheduler.setSchedulingAgent(SchedulingStrategy.TIMER_DRIVEN, Mockito.mock(SchedulingAgent.class));
reportingTask = new TestReportingTask();
@ -666,6 +667,7 @@ public class TestStandardProcessScheduler {
}
private StandardProcessScheduler createScheduler() {
return new StandardProcessScheduler(new FlowEngine(1, "Unit Test", true), Mockito.mock(FlowController.class), stateMgrProvider, nifiProperties);
return new StandardProcessScheduler(new FlowEngine(1, "Unit Test", true), Mockito.mock(FlowController.class),
stateMgrProvider, nifiProperties, new StandardLifecycleStateManager());
}
}

View File

@ -17,15 +17,6 @@
package org.apache.nifi.controller.service;
import static org.junit.jupiter.api.Assertions.assertTrue;
import java.beans.PropertyDescriptor;
import java.util.Collections;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ExecutionException;
import org.apache.nifi.bundle.Bundle;
import org.apache.nifi.bundle.BundleCoordinate;
import org.apache.nifi.components.state.StateManager;
@ -37,6 +28,7 @@ import org.apache.nifi.controller.NodeTypeProvider;
import org.apache.nifi.controller.ProcessScheduler;
import org.apache.nifi.controller.ReloadComponent;
import org.apache.nifi.controller.flow.FlowManager;
import org.apache.nifi.controller.scheduling.StandardLifecycleStateManager;
import org.apache.nifi.controller.scheduling.StandardProcessScheduler;
import org.apache.nifi.controller.service.mock.MockProcessGroup;
import org.apache.nifi.controller.service.mock.ServiceA;
@ -53,6 +45,16 @@ import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
import org.mockito.Mockito;
import java.beans.PropertyDescriptor;
import java.util.Collections;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ExecutionException;
import static org.junit.jupiter.api.Assertions.assertTrue;
public class StandardControllerServiceProviderIT {
private static Bundle systemBundle;
private static NiFiProperties niFiProperties;
@ -103,7 +105,7 @@ public class StandardControllerServiceProviderIT {
@Timeout(120)
public void testConcurrencyWithEnablingReferencingServicesGraph() throws InterruptedException, ExecutionException {
final StandardProcessScheduler scheduler = new StandardProcessScheduler(new FlowEngine(1, "Unit Test", true), Mockito.mock(FlowController.class),
stateManagerProvider, niFiProperties);
stateManagerProvider, niFiProperties, new StandardLifecycleStateManager());
for (int i = 0; i < 5000; i++) {
testEnableReferencingServicesGraph(scheduler);

View File

@ -36,6 +36,7 @@ import org.apache.nifi.controller.ReloadComponent;
import org.apache.nifi.controller.ScheduledState;
import org.apache.nifi.controller.StandardProcessorNode;
import org.apache.nifi.controller.flow.FlowManager;
import org.apache.nifi.controller.scheduling.StandardLifecycleStateManager;
import org.apache.nifi.controller.scheduling.StandardProcessScheduler;
import org.apache.nifi.controller.service.mock.DummyProcessor;
import org.apache.nifi.controller.service.mock.MockProcessGroup;
@ -149,7 +150,7 @@ public class TestStandardControllerServiceProvider {
private StandardProcessScheduler createScheduler() {
return new StandardProcessScheduler(new FlowEngine(1, "Unit Test", true), Mockito.mock(FlowController.class),
stateManagerProvider, niFiProperties);
stateManagerProvider, niFiProperties, new StandardLifecycleStateManager());
}
private void setProperty(ControllerServiceNode serviceNode, String propName, String propValue) {

View File

@ -34,6 +34,7 @@ import org.apache.nifi.controller.label.Label;
import org.apache.nifi.controller.queue.DropFlowFileStatus;
import org.apache.nifi.controller.queue.QueueSize;
import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.flow.ExecutionEngine;
import org.apache.nifi.flow.VersionedExternalFlow;
import org.apache.nifi.groups.BatchCounts;
import org.apache.nifi.groups.DataValve;
@ -45,6 +46,8 @@ import org.apache.nifi.groups.NoOpBatchCounts;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.groups.ProcessGroupCounts;
import org.apache.nifi.groups.RemoteProcessGroup;
import org.apache.nifi.groups.StatelessGroupNode;
import org.apache.nifi.groups.StatelessGroupScheduledState;
import org.apache.nifi.parameter.ParameterContext;
import org.apache.nifi.parameter.ParameterUpdate;
import org.apache.nifi.registry.VariableRegistry;
@ -69,6 +72,8 @@ import java.util.function.Predicate;
public class MockProcessGroup implements ProcessGroup {
private final Map<String, ControllerServiceNode> serviceMap = new HashMap<>();
private final Map<String, ProcessorNode> processorMap = new HashMap<>();
private final Map<String, Port> inputPortMap = new HashMap<>();
private final Map<String, Port> outputPortMap = new HashMap<>();
private final FlowManager flowManager;
private final MutableVariableRegistry variableRegistry = new MutableVariableRegistry(VariableRegistry.ENVIRONMENT_SYSTEM_REGISTRY);
private VersionControlInformation versionControlInfo;
@ -148,12 +153,44 @@ public class MockProcessGroup implements ProcessGroup {
@Override
public void startProcessing() {
}
@Override
public void stopProcessing() {
public void startComponents() {
}
@Override
public CompletableFuture<Void> stopProcessing() {
return CompletableFuture.completedFuture(null);
}
@Override
public CompletableFuture<Void> stopComponents() {
return CompletableFuture.completedFuture(null);
}
@Override
public StatelessGroupScheduledState getStatelessScheduledState() {
return StatelessGroupScheduledState.STOPPED;
}
@Override
public StatelessGroupScheduledState getDesiredStatelessScheduledState() {
return StatelessGroupScheduledState.STOPPED;
}
@Override
public boolean isStatelessActive() {
return false;
}
@Override
public void setExecutionEngine(final ExecutionEngine executionEngine) {
}
@Override
public Optional<StatelessGroupNode> getStatelessGroupNode() {
return Optional.empty();
}
@Override
@ -171,11 +208,6 @@ public class MockProcessGroup implements ProcessGroup {
}
@Override
public void enableAllControllerServices() {
}
@Override
public CompletableFuture<Void> startProcessor(final ProcessorNode processor, final boolean failIfStopping) {
return CompletableFuture.completedFuture(null);
@ -243,42 +275,56 @@ public class MockProcessGroup implements ProcessGroup {
@Override
public void addInputPort(final Port port) {
port.setProcessGroup(this);
inputPortMap.put(port.getIdentifier(), port);
if (flowManager != null) {
flowManager.onInputPortAdded(port);
}
}
@Override
public void removeInputPort(final Port port) {
inputPortMap.remove(port.getIdentifier());
if (flowManager != null) {
flowManager.onInputPortRemoved(port);
}
}
@Override
public Set<Port> getInputPorts() {
return null;
return new HashSet<>(inputPortMap.values());
}
@Override
public Port getInputPort(final String id) {
return null;
return inputPortMap.get(id);
}
@Override
public void addOutputPort(final Port port) {
port.setProcessGroup(this);
outputPortMap.put(port.getIdentifier(), port);
if (flowManager != null) {
flowManager.onInputPortAdded(port);
}
}
@Override
public void removeOutputPort(final Port port) {
outputPortMap.remove(port.getIdentifier());
if (flowManager != null) {
flowManager.onInputPortRemoved(port);
}
}
@Override
public Port getOutputPort(final String id) {
return null;
return outputPortMap.get(id);
}
@Override
public Set<Port> getOutputPorts() {
return null;
return new HashSet<>(outputPortMap.values());
}
@Override
@ -614,6 +660,10 @@ public class MockProcessGroup implements ProcessGroup {
}
@Override
public void verifyCanScheduleComponentsIndividually() {
}
@Override
public void verifyCanStop() {
@ -863,6 +913,37 @@ public class MockProcessGroup implements ProcessGroup {
return null;
}
public ExecutionEngine getExecutionEngine() {
return ExecutionEngine.STANDARD;
}
@Override
public ExecutionEngine resolveExecutionEngine() {
return ExecutionEngine.STANDARD;
}
@Override
public void verifyCanSetExecutionEngine(final ExecutionEngine executionEngine) {
}
@Override
public void setMaxConcurrentTasks(final int maxConcurrentTasks) {
}
@Override
public int getMaxConcurrentTasks() {
return 0;
}
@Override
public void setStatelessFlowTimeout(final String timeout) {
}
@Override
public String getStatelessFlowTimeout() {
return null;
}
@Override
public void setLogFileSuffix(String logFileSuffix) {
@ -871,4 +952,5 @@ public class MockProcessGroup implements ProcessGroup {
@Override
public void terminateProcessor(ProcessorNode processor) {
}
}

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