diff --git a/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/EventDriven.java b/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/EventDriven.java new file mode 100644 index 0000000000..279a49eadb --- /dev/null +++ b/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/EventDriven.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.annotation.behavior; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Inherited; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + *

+ * Annotation that may be placed on a Processor that indicates to the framework + * that the Processor is eligible to be scheduled to run based on the occurrence + * of an "Event" (e.g., when a FlowFile is enqueued in an incoming Connection), + * rather than being triggered periodically. + *

+ * + *

+ * This Annotation should not be used in conjunction with + * {@link TriggerSerially} or {@link TriggerWhenEmpty}. If this Annotation is + * used with either of these other Annotations, the Processor will not be + * eligible to be scheduled in Event-Driven mode. + *

+ * + * @author none + */ +@Documented +@Target({ElementType.TYPE}) +@Retention(RetentionPolicy.RUNTIME) +@Inherited +public @interface EventDriven { + +} diff --git a/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/SideEffectFree.java b/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/SideEffectFree.java new file mode 100644 index 0000000000..f32acc37db --- /dev/null +++ b/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/SideEffectFree.java @@ -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.annotation.behavior; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Inherited; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Marker annotation a {@link org.apache.nifi.processor.Processor Processor} + * implementation can use to indicate that its + * operations on FlowFiles can be safely repeated across process sessions. If a + * processor has this annotation and it allows the framework to manage session + * commit and rollback then the framework may elect to cascade a + * {@link org.apache.nifi.processor.ProcessSession ProcessSession} given to this + * processor's onTrigger method to the + * onTrigger method of another processor. It can do this knowing that if + * something fails along a series of processors using this same session that it + * can all be safely rolled back without any ill effects on external services + * which could not be rolled back and thus all the processes could be safely + * repeated (implied idempotent behavior). + * + * @author none + */ +@Documented +@Target({ElementType.TYPE}) +@Retention(RetentionPolicy.RUNTIME) +@Inherited +public @interface SideEffectFree { +} diff --git a/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/SupportsBatching.java b/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/SupportsBatching.java new file mode 100644 index 0000000000..f5fd61f0df --- /dev/null +++ b/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/SupportsBatching.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.annotation.behavior; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Inherited; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Marker annotation a Processor implementation can use to indicate that users + * should be able to supply a Batch Duration for the Processor. If a Processor + * uses this annotation, it is allowing the Framework to batch + * {@link nifi.processor.ProcessSession ProcessSession}s' commits, as well as + * allowing the Framework to return the same ProcessSession multiple times from + * subsequent calls to + * {@link nifi.processor.ProcessSessionFactory ProcessSessionFactory}. + * {@link nifi.processor.ProcessSessionFactory#createSession() createSession()}. + * + * When this Annotation is used, it is important to note that calls to + * {@link nifi.processor.ProcessSession#commit() ProcessSession.commit()} may + * not provide a guarantee that the data has been safely stored in NiFi's + * Content Repository or FlowFile Repository. Therefore, it is not appropriate, + * for instance, to use this annotation if the Processor will call + * ProcessSession.commit() to ensure data is persisted before deleting the data + * from a remote source. + * + * @author none + */ +@Documented +@Target({ElementType.TYPE}) +@Retention(RetentionPolicy.RUNTIME) +@Inherited +public @interface SupportsBatching { + +} diff --git a/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/TriggerSerially.java b/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/TriggerSerially.java new file mode 100644 index 0000000000..7bf7d0b62f --- /dev/null +++ b/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/TriggerSerially.java @@ -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.annotation.behavior; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Inherited; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Marker annotation a {@link org.apache.nifi.processor.Processor Processor} + * implementation can use to indicate that the + * Processor is not safe for concurrent execution of its onTrigger() + * method. By default, Processors are assumed to be safe for concurrent + * execution. + * + * @author none + */ +@Documented +@Target({ElementType.TYPE}) +@Retention(RetentionPolicy.RUNTIME) +@Inherited +public @interface TriggerSerially { +} diff --git a/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/TriggerWhenAnyDestinationAvailable.java b/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/TriggerWhenAnyDestinationAvailable.java new file mode 100644 index 0000000000..803aa2f904 --- /dev/null +++ b/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/TriggerWhenAnyDestinationAvailable.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.annotation.behavior; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Inherited; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Marker annotation a {@link org.apache.nifi.processor.Processor Processor} + * implementation can use to indicate that the + * Processor is to be triggered if any of its destinations has available space + * for incoming FlowFiles. By default, Processors are triggered only when all + * destinations report that they have available space (i.e., none of the outgoing + * Connections is full). + * + * @author none + */ +@Documented +@Target({ElementType.TYPE}) +@Retention(RetentionPolicy.RUNTIME) +@Inherited +public @interface TriggerWhenAnyDestinationAvailable { + +} diff --git a/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/TriggerWhenEmpty.java b/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/TriggerWhenEmpty.java new file mode 100644 index 0000000000..fed9b34823 --- /dev/null +++ b/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/TriggerWhenEmpty.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.annotation.behavior; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Inherited; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Marker annotation a {@link org.apache.nifi.processor.Processor Processor} + * implementation can use to indicate that the + * Processor should still be triggered even when it has no data in its work + * queue. By default, Processors which have no non-self incoming edges will be + * triggered even if there is no work in its queue. However, Processors that + * have non-self incoming edges will only be triggered if they have work in + * their queue or they present this annotation. + * + * @author none + */ +@Documented +@Target({ElementType.TYPE}) +@Retention(RetentionPolicy.RUNTIME) +@Inherited +public @interface TriggerWhenEmpty { +} diff --git a/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/CapabilityDescription.java b/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/CapabilityDescription.java new file mode 100644 index 0000000000..d69788a9b1 --- /dev/null +++ b/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/CapabilityDescription.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.annotation.documentation; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Inherited; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Annotation that may be placed on a {@link org.apache.nifi.processor.Processor Processor}, + * {@link org.apache.nifi.controller.ControllerService ControllerService}, or + * {@link org.apache.nifi.reporting.ReportingTask ReportingTask} allowing for a description to be + * provided. This description can be provided to a user in logs, UI, etc. + * + * @author none + */ +@Documented +@Target({ElementType.TYPE}) +@Retention(RetentionPolicy.RUNTIME) +@Inherited +public @interface CapabilityDescription { + + String value(); +} diff --git a/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/Tags.java b/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/Tags.java new file mode 100644 index 0000000000..8bd8f9a051 --- /dev/null +++ b/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/Tags.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.annotation.documentation; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Inherited; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Annotation that can be applied to a {@link org.apache.nifi.processor.Processor Processor}, + * {@link org.apache.nifi.controller.ControllerService ControllerService}, or + * {@link org.apache.nifi.reporting.ReportingTask ReportingTask} in order to associate + * tags (keywords) with the component. These tags do not affect the component in + * any way but serve as additional documentation and can be used to sort/filter + * Processors. + * + * @author none + */ +@Documented +@Target({ElementType.TYPE}) +@Retention(RetentionPolicy.RUNTIME) +@Inherited +public @interface Tags { + + /** + * @return all tag values associated with the given processor + */ + public String[] value(); +} diff --git a/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnAdded.java b/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnAdded.java new file mode 100644 index 0000000000..acb7a4d027 --- /dev/null +++ b/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnAdded.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.annotation.lifecycle; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Inherited; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Marker annotation a {@link org.apache.nifi.processor.Processor Processor}, + * {@link org.apache.nifi.controller.ControllerService ControllerService}, or + * {@link org.apache.nifi.reporting.ReportingTask ReportingTask} + * implementation can use to indicate a method + * should be called whenever the component is added to the flow. This method + * will be called once for the entire life of a component instance. + * + * If any method annotated with this annotation throws a Throwable, the component + * will not be added to the flow. + * + * @author none + */ +@Documented +@Target({ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +@Inherited +public @interface OnAdded { +} diff --git a/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnRemoved.java b/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnRemoved.java new file mode 100644 index 0000000000..696159f0e4 --- /dev/null +++ b/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnRemoved.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.annotation.lifecycle; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Inherited; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Marker annotation a {@link org.apache.nifi.processor.Processor Processor}, + * {@link org.apache.nifi.controller.ControllerService ControllerService}, or + * {@link org.apache.nifi.reporting.ReportingTask ReportingTask} implementation + * can use to indicate a method should be called whenever the component is removed + * from the flow. This method will be called once for the entire life of a + * component instance. If the method throw any Throwable, that Throwable will be + * caught and logged but will not prevent subsequent methods with this annotation + * or removal of the component from the flow. + * + * @author none + */ +@Documented +@Target({ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +@Inherited +public @interface OnRemoved { +} diff --git a/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnScheduled.java b/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnScheduled.java new file mode 100644 index 0000000000..9dfd150526 --- /dev/null +++ b/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnScheduled.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.annotation.lifecycle; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Inherited; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Marker annotation a {@link org.apache.nifi.processor.Processor Processor} or + * {@link org.apache.nifi.reporting.ReportingTask ReportingTask} implementation + * can use to indicate a method should be called whenever the component is scheduled + * to run. This will be called before any call to 'onTrigger' and will be called once each time + * a Processor or Reporting Task is scheduled to run. + * + *

+ * Methods using this annotation must take either 0 arguments or a single argument. + *

+ * + *

+ * If using 1 argument and the component using the annotation is a Processor, that argument must + * be of type {@link org.apache.nifi.processor.ProcessContext ProcessContext}. + *

+ * + *

+ * If using 1 argument and the component using the annotation is a Reporting Task, that argument must + * be of type {@link org.apache.nifi.reporting.ReportingContext ReportingContext}. + *

+ * + * If any method annotated with this annotation throws any Throwable, the framework will wait a while + * and then attempt to invoke the method again. This will continue until the method succeeds, and the + * component will then be scheduled to run after this method return successfully. + * + * @author none + */ +@Documented +@Target({ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +@Inherited +public @interface OnScheduled { +} diff --git a/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnShutdown.java b/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnShutdown.java new file mode 100644 index 0000000000..a4129e158d --- /dev/null +++ b/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnShutdown.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.annotation.lifecycle; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Inherited; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Marker annotation a {@link org.apache.nifi.processor.Processor Processor}, + * {@link org.apache.nifi.controller.ControllerService ControllerService}, or + * {@link org.apache.nifi.reporting.ReportingTask ReportingTask} implementation + * can use to indicate a method should be called whenever the flow is being shutdown. + * This will be called at most once for each component in a JVM lifetime. + * It is not, however, guaranteed that this method will be called on shutdown, as + * the service may be killed suddenly. + * + * @author none + */ +@Documented +@Target({ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +@Inherited +public @interface OnShutdown { +} diff --git a/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnStopped.java b/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnStopped.java new file mode 100644 index 0000000000..4715253234 --- /dev/null +++ b/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnStopped.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.annotation.lifecycle; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Inherited; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + *

+ * Marker annotation a {@link org.apache.nifi.processor.Processor Processor} or + * {@link org.apache.nifi.reporting.ReportingTask ReportingTask} + * implementation can use to indicate that a method + * should be called whenever the component is no longer scheduled to run. + * Methods marked with this annotation will be invoked each time the component + * is stopped and will be invoked only after the last thread has returned from + * the onTrigger method. + *

+ * + *

+ * This means that the thread executing in this method will be the only thread + * executing in any part of the Processor. However, since other threads may + * later execute other parts of the code, member variables must still be + * protected appropriately. However, access to multiple variables need not be + * atomic. + *

+ * + *

+ * To indicate that a method should be called immediately when a component is no + * longer scheduled to run (as opposed to after all threads have returned from the + * onTrigger method), see the {@link OnUnscheduled} annotation. + *

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

+ * Marker annotation a {@link org.apache.nifi.processor.Processor Processor} or + * {@link org.apache.nifi.reporting.ReportingTask ReportingTask} + * should be called whenever the component is no longer scheduled to run. + * Methods marked with this annotation will be invoked each time the framework + * is notified to stop scheduling the component. This method is invoked as other + * threads are potentially running. To invoke a method after all threads have + * finished processing, see the {@link OnStopped} annotation. + *

+ * + * @author none + */ +@Documented +@Target({ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +@Inherited +public @interface OnUnscheduled { +} diff --git a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowController.java index 346e801660..860ea2dc9f 100644 --- a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowController.java +++ b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/FlowController.java @@ -49,6 +49,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import javax.net.ssl.SSLContext; import org.apache.nifi.admin.service.UserService; +import org.apache.nifi.annotation.lifecycle.OnAdded; import org.apache.nifi.cluster.BulletinsPayload; import org.apache.nifi.cluster.HeartbeatPayload; import org.apache.nifi.cluster.protocol.DataFlow; @@ -142,7 +143,6 @@ import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.SimpleProcessLogger; import org.apache.nifi.processor.StandardProcessorInitializationContext; import org.apache.nifi.processor.StandardValidationContextFactory; -import org.apache.nifi.processor.annotation.OnAdded; import org.apache.nifi.provenance.ProvenanceEventRecord; import org.apache.nifi.provenance.ProvenanceEventRepository; import org.apache.nifi.provenance.ProvenanceEventType; @@ -785,6 +785,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H * @throws ProcessorInstantiationException if the processor cannot be * instantiated for any reason */ + @SuppressWarnings("deprecation") public ProcessorNode createProcessor(final String type, String id, final boolean firstTimeAdded) throws ProcessorInstantiationException { id = id.intern(); final Processor processor = instantiateProcessor(type, id); @@ -796,7 +797,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H if ( firstTimeAdded ) { try (final NarCloseable x = NarCloseable.withNarLoader()) { - ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, processor); + ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, org.apache.nifi.processor.annotation.OnAdded.class, processor); } catch (final Exception e) { logRepository.removeObserver(StandardProcessorNode.BULLETIN_OBSERVER_ID); throw new ProcessorLifeCycleException("Failed to invoke @OnAdded methods of " + procNode.getProcessor(), e); diff --git a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java index 071be4d948..fe72ae45ca 100644 --- a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java +++ b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java @@ -16,9 +16,17 @@ */ package org.apache.nifi.controller; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.behavior.TriggerSerially; +import org.apache.nifi.annotation.behavior.TriggerWhenAnyDestinationAvailable; +import org.apache.nifi.annotation.behavior.TriggerWhenEmpty; +import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.controller.ProcessScheduler; import org.apache.nifi.controller.ValidationContextFactory; import org.apache.nifi.controller.ProcessorNode; + import static java.util.Objects.requireNonNull; import java.util.ArrayList; @@ -53,16 +61,8 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSessionFactory; import org.apache.nifi.processor.Processor; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.EventDriven; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.TriggerSerially; -import org.apache.nifi.processor.annotation.TriggerWhenAnyDestinationAvailable; -import org.apache.nifi.processor.annotation.TriggerWhenEmpty; import org.apache.nifi.scheduling.SchedulingStrategy; import org.apache.nifi.util.FormatUtils; - import org.apache.commons.lang3.builder.EqualsBuilder; import org.apache.commons.lang3.builder.HashCodeBuilder; import org.quartz.CronExpression; @@ -119,6 +119,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable private SchedulingStrategy schedulingStrategy; // guarded by read/write lock + @SuppressWarnings("deprecation") StandardProcessorNode(final Processor processor, final String uuid, final ValidationContextFactory validationContextFactory, final ProcessScheduler scheduler, final ControllerServiceProvider controllerServiceProvider) { super(processor, uuid, validationContextFactory, controllerServiceProvider); @@ -150,13 +151,14 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable isolated = new AtomicBoolean(false); penalizationPeriod = new AtomicReference<>(DEFAULT_PENALIZATION_PERIOD); - triggerWhenEmpty = processor.getClass().isAnnotationPresent(TriggerWhenEmpty.class); - sideEffectFree = processor.getClass().isAnnotationPresent(SideEffectFree.class); - batchSupported = processor.getClass().isAnnotationPresent(SupportsBatching.class); - triggeredSerially = processor.getClass().isAnnotationPresent(TriggerSerially.class); - triggerWhenAnyDestinationAvailable = processor.getClass().isAnnotationPresent(TriggerWhenAnyDestinationAvailable.class); + final Class procClass = processor.getClass(); + triggerWhenEmpty = procClass.isAnnotationPresent(TriggerWhenEmpty.class) || procClass.isAnnotationPresent(org.apache.nifi.processor.annotation.TriggerWhenEmpty.class); + sideEffectFree = procClass.isAnnotationPresent(SideEffectFree.class) || procClass.isAnnotationPresent(org.apache.nifi.processor.annotation.SideEffectFree.class); + batchSupported = procClass.isAnnotationPresent(SupportsBatching.class) || procClass.isAnnotationPresent(org.apache.nifi.processor.annotation.SupportsBatching.class); + triggeredSerially = procClass.isAnnotationPresent(TriggerSerially.class) || procClass.isAnnotationPresent(org.apache.nifi.processor.annotation.TriggerSerially.class); + triggerWhenAnyDestinationAvailable = procClass.isAnnotationPresent(TriggerWhenAnyDestinationAvailable.class) || procClass.isAnnotationPresent(org.apache.nifi.processor.annotation.TriggerWhenAnyDestinationAvailable.class); this.validationContextFactory = validationContextFactory; - eventDrivenSupported = processor.getClass().isAnnotationPresent(EventDriven.class) && !triggeredSerially && !triggerWhenEmpty; + eventDrivenSupported = (procClass.isAnnotationPresent(EventDriven.class) || procClass.isAnnotationPresent(org.apache.nifi.processor.annotation.EventDriven.class) )&& !triggeredSerially && !triggerWhenEmpty; schedulingStrategy = SchedulingStrategy.TIMER_DRIVEN; } @@ -354,9 +356,21 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable * @return the value of the processor's {@link CapabilityDescription} * annotation, if one exists, else null. */ + @SuppressWarnings("deprecation") public String getProcessorDescription() { - final CapabilityDescription capDesc = processor.getClass().getAnnotation(CapabilityDescription.class); - return (capDesc == null) ? null : capDesc.value(); + CapabilityDescription capDesc = processor.getClass().getAnnotation(CapabilityDescription.class); + String description = null; + if ( capDesc != null ) { + description = capDesc.value(); + } else { + final org.apache.nifi.processor.annotation.CapabilityDescription deprecatedCapDesc = + processor.getClass().getAnnotation(org.apache.nifi.processor.annotation.CapabilityDescription.class); + if ( deprecatedCapDesc != null ) { + description = deprecatedCapDesc.value(); + } + } + + return description; } @Override diff --git a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java index af801bbb8c..7455bf8485 100644 --- a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java +++ b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java @@ -23,6 +23,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; +import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.controller.EventBasedWorker; import org.apache.nifi.controller.EventDrivenWorkerQueue; @@ -41,12 +42,10 @@ import org.apache.nifi.nar.NarCloseable; import org.apache.nifi.processor.ProcessSessionFactory; import org.apache.nifi.processor.SimpleProcessLogger; import org.apache.nifi.processor.StandardProcessContext; -import org.apache.nifi.processor.annotation.OnStopped; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.util.Connectables; import org.apache.nifi.util.FormatUtils; import org.apache.nifi.util.ReflectionUtils; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -262,6 +261,7 @@ public class EventDrivenSchedulingAgent implements SchedulingAgent { } } + @SuppressWarnings("deprecation") private void trigger(final Connectable worker, final ScheduleState scheduleState, final ConnectableProcessContext processContext, final ProcessSessionFactory sessionFactory) { final int newThreadCount = scheduleState.incrementActiveThreadCount(); if (newThreadCount > worker.getMaxConcurrentTasks() && worker.getMaxConcurrentTasks() > 0) { @@ -294,7 +294,7 @@ public class EventDrivenSchedulingAgent implements SchedulingAgent { } finally { if (!scheduleState.isScheduled() && scheduleState.getActiveThreadCount() == 1 && scheduleState.mustCallOnStoppedMethods()) { try (final NarCloseable x = NarCloseable.withNarLoader()) { - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, worker, processContext); + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, org.apache.nifi.processor.annotation.OnStopped.class, worker, processContext); } } diff --git a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java index 7fc65f9426..5950b4e819 100644 --- a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java +++ b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java @@ -27,6 +27,9 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.annotation.lifecycle.OnUnscheduled; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.Funnel; import org.apache.nifi.connectable.Port; @@ -47,15 +50,11 @@ import org.apache.nifi.processor.SchedulingContext; import org.apache.nifi.processor.SimpleProcessLogger; import org.apache.nifi.processor.StandardProcessContext; import org.apache.nifi.processor.StandardSchedulingContext; -import org.apache.nifi.processor.annotation.OnScheduled; -import org.apache.nifi.processor.annotation.OnStopped; -import org.apache.nifi.processor.annotation.OnUnscheduled; import org.apache.nifi.reporting.ReportingTask; import org.apache.nifi.scheduling.SchedulingStrategy; import org.apache.nifi.util.FormatUtils; import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.util.ReflectionUtils; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -209,6 +208,7 @@ public final class StandardProcessScheduler implements ProcessScheduler { scheduleState.setScheduled(false); final Runnable unscheduleReportingTaskRunnable = new Runnable() { + @SuppressWarnings("deprecation") @Override public void run() { final ConfigurationContext configurationContext = taskNode.getConfigurationContext(); @@ -216,7 +216,7 @@ public final class StandardProcessScheduler implements ProcessScheduler { while (true) { try { try (final NarCloseable x = NarCloseable.withNarLoader()) { - ReflectionUtils.invokeMethodsWithAnnotation(OnUnscheduled.class, reportingTask, configurationContext); + ReflectionUtils.invokeMethodsWithAnnotation(OnUnscheduled.class, org.apache.nifi.processor.annotation.OnUnscheduled.class, reportingTask, configurationContext); } break; } catch (final InvocationTargetException ite) { @@ -241,7 +241,7 @@ public final class StandardProcessScheduler implements ProcessScheduler { agent.unschedule(taskNode, scheduleState); if (scheduleState.getActiveThreadCount() == 0) { - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, reportingTask, configurationContext); + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, org.apache.nifi.processor.annotation.OnStopped.class, reportingTask, configurationContext); } } }; @@ -276,6 +276,7 @@ public final class StandardProcessScheduler implements ProcessScheduler { } final Runnable startProcRunnable = new Runnable() { + @SuppressWarnings("deprecation") @Override public void run() { try (final NarCloseable x = NarCloseable.withNarLoader()) { @@ -297,7 +298,7 @@ public final class StandardProcessScheduler implements ProcessScheduler { } final SchedulingContext schedulingContext = new StandardSchedulingContext(processContext, controllerServiceProvider, procNode); - ReflectionUtils.invokeMethodsWithAnnotation(OnScheduled.class, procNode.getProcessor(), schedulingContext); + ReflectionUtils.invokeMethodsWithAnnotation(OnScheduled.class, org.apache.nifi.processor.annotation.OnScheduled.class, procNode.getProcessor(), schedulingContext); getSchedulingAgent(procNode).schedule(procNode, scheduleState); diff --git a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java index c04a04fa76..aca870b1cb 100644 --- a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java +++ b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java @@ -18,6 +18,7 @@ package org.apache.nifi.controller.tasks; import java.util.concurrent.atomic.AtomicLong; +import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.controller.repository.StandardProcessSessionFactory; import org.apache.nifi.controller.scheduling.ConnectableProcessContext; @@ -26,11 +27,9 @@ import org.apache.nifi.controller.scheduling.ScheduleState; import org.apache.nifi.encrypt.StringEncryptor; import org.apache.nifi.nar.NarCloseable; import org.apache.nifi.processor.ProcessSessionFactory; -import org.apache.nifi.processor.annotation.OnStopped; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.util.Connectables; import org.apache.nifi.util.ReflectionUtils; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -50,6 +49,7 @@ public class ContinuallyRunConnectableTask implements Runnable { this.processContext = new ConnectableProcessContext(connectable, encryptor); } + @SuppressWarnings("deprecation") @Override public void run() { if (!scheduleState.isScheduled()) { @@ -86,7 +86,7 @@ public class ContinuallyRunConnectableTask implements Runnable { } finally { if (!scheduleState.isScheduled() && scheduleState.getActiveThreadCount() == 1 && scheduleState.mustCallOnStoppedMethods()) { try (final NarCloseable x = NarCloseable.withNarLoader()) { - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, connectable, processContext); + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, org.apache.nifi.processor.annotation.OnStopped.class, connectable, processContext); } } diff --git a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunProcessorTask.java b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunProcessorTask.java index 65c375fa34..33bd3279a1 100644 --- a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunProcessorTask.java +++ b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunProcessorTask.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; +import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.controller.FlowController; import org.apache.nifi.controller.ProcessorNode; import org.apache.nifi.controller.repository.BatchingSessionFactory; @@ -36,11 +37,9 @@ import org.apache.nifi.nar.NarCloseable; import org.apache.nifi.processor.ProcessSessionFactory; import org.apache.nifi.processor.SimpleProcessLogger; import org.apache.nifi.processor.StandardProcessContext; -import org.apache.nifi.processor.annotation.OnStopped; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.util.Connectables; import org.apache.nifi.util.ReflectionUtils; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -69,6 +68,7 @@ public class ContinuallyRunProcessorTask implements Runnable { this.processContext = new StandardProcessContext(procNode, flowController, encryptor); } + @SuppressWarnings("deprecation") @Override public void run() { // make sure processor is not yielded @@ -163,7 +163,7 @@ public class ContinuallyRunProcessorTask implements Runnable { // invoke the OnStopped methods if (!scheduleState.isScheduled() && scheduleState.getActiveThreadCount() == 1 && scheduleState.mustCallOnStoppedMethods()) { try (final NarCloseable x = NarCloseable.withNarLoader()) { - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, procNode.getProcessor(), processContext); + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, org.apache.nifi.processor.annotation.OnStopped.class, procNode.getProcessor(), processContext); flowController.heartbeat(); } } diff --git a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java index 36aa9dd31e..9b7058101f 100644 --- a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java +++ b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java @@ -16,12 +16,11 @@ */ package org.apache.nifi.controller.tasks; +import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.controller.ReportingTaskNode; import org.apache.nifi.controller.scheduling.ScheduleState; import org.apache.nifi.nar.NarCloseable; -import org.apache.nifi.processor.annotation.OnStopped; import org.apache.nifi.util.ReflectionUtils; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,6 +36,7 @@ public class ReportingTaskWrapper implements Runnable { this.scheduleState = scheduleState; } + @SuppressWarnings("deprecation") @Override public synchronized void run() { scheduleState.incrementActiveThreadCount(); @@ -52,7 +52,7 @@ public class ReportingTaskWrapper implements Runnable { // invoke the OnStopped methods if (!scheduleState.isScheduled() && scheduleState.getActiveThreadCount() == 1 && scheduleState.mustCallOnStoppedMethods()) { try (final NarCloseable x = NarCloseable.withNarLoader()) { - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, taskNode.getReportingTask(), taskNode.getReportingContext()); + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, org.apache.nifi.processor.annotation.OnStopped.class, taskNode.getReportingTask(), taskNode.getReportingContext()); } } diff --git a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java index 8cff5dd2dc..856ccc1273 100644 --- a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java +++ b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java @@ -30,6 +30,8 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import org.apache.nifi.annotation.lifecycle.OnRemoved; +import org.apache.nifi.annotation.lifecycle.OnShutdown; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.ConnectableType; import org.apache.nifi.connectable.Connection; @@ -46,10 +48,7 @@ import org.apache.nifi.controller.label.Label; import org.apache.nifi.controller.service.ControllerServiceProvider; import org.apache.nifi.logging.LogRepositoryFactory; import org.apache.nifi.nar.NarCloseable; -import org.apache.nifi.processor.SimpleProcessLogger; import org.apache.nifi.processor.StandardProcessContext; -import org.apache.nifi.processor.annotation.OnRemoved; -import org.apache.nifi.processor.annotation.OnShutdown; import org.apache.nifi.remote.RemoteGroupPort; import org.apache.nifi.remote.RootGroupPort; import org.apache.nifi.util.NiFiProperties; @@ -326,10 +325,11 @@ public final class StandardProcessGroup implements ProcessGroup { } } + @SuppressWarnings("deprecation") private void shutdown(final ProcessGroup procGroup) { for (final ProcessorNode node : procGroup.getProcessors()) { try (final NarCloseable x = NarCloseable.withNarLoader()) { - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, node.getProcessor()); + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, org.apache.nifi.processor.annotation.OnShutdown.class, node.getProcessor()); } } @@ -652,6 +652,7 @@ public final class StandardProcessGroup implements ProcessGroup { } } + @SuppressWarnings("deprecation") @Override public void removeProcessor(final ProcessorNode processor) { final String id = requireNonNull(processor).getIdentifier(); @@ -668,7 +669,7 @@ public final class StandardProcessGroup implements ProcessGroup { try (final NarCloseable x = NarCloseable.withNarLoader()) { final StandardProcessContext processContext = new StandardProcessContext(processor, controllerServiceProvider, encryptor); - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, processor.getProcessor(), processContext); + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, org.apache.nifi.processor.annotation.OnRemoved.class, processor.getProcessor(), processContext); } catch (final Exception e) { throw new ProcessorLifeCycleException("Failed to invoke 'OnRemoved' methods of " + processor, e); } diff --git a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/util/ReflectionUtils.java b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/util/ReflectionUtils.java index e15e00a6bc..f8e7da46c0 100644 --- a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/util/ReflectionUtils.java +++ b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/util/ReflectionUtils.java @@ -19,6 +19,9 @@ package org.apache.nifi.util; import java.lang.annotation.Annotation; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.List; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -42,54 +45,92 @@ public class ReflectionUtils { * @throws IllegalAccessException */ public static void invokeMethodsWithAnnotation(final Class annotation, final Object instance, final Object... args) throws IllegalAccessException, IllegalArgumentException, InvocationTargetException { - try { - for (final Method method : instance.getClass().getMethods()) { - if (method.isAnnotationPresent(annotation)) { - final boolean isAccessible = method.isAccessible(); - method.setAccessible(true); + invokeMethodsWithAnnotation(annotation, null, instance, args); + } + - try { - final Class[] argumentTypes = method.getParameterTypes(); - if (argumentTypes.length > args.length) { - throw new IllegalArgumentException(String.format("Unable to invoke method %1$s on %2$s because method expects %3$s parameters but only %4$s were given", - method.getName(), instance, argumentTypes.length, args.length)); - } - - for (int i = 0; i < argumentTypes.length; i++) { - final Class argType = argumentTypes[i]; - if (!argType.isAssignableFrom(args[i].getClass())) { - throw new IllegalArgumentException(String.format( - "Unable to invoke method %1$s on %2$s because method parameter %3$s is expected to be of type %4$s but argument passed was of type %5$s", - method.getName(), instance, i, argType, args[i].getClass())); + /** + * Invokes all methods on the given instance that have been annotated with + * the given preferredAnnotation and if no such method exists will invoke all + * methods on the given instance that have been annotated with the given + * alternateAnnotation, if any exists. If the signature of the method that is defined in + * instance uses 1 or more parameters, those parameters must be + * specified by the args parameter. However, if more arguments + * are supplied by the args parameter than needed, the extra + * arguments will be ignored. + * + * @param preferredAnnotation + * @param alternateAnnotation + * @param instance + * @param args + * @throws InvocationTargetException + * @throws IllegalArgumentException + * @throws IllegalAccessException + */ + public static void invokeMethodsWithAnnotation(final Class preferredAnnotation, final Class alternateAnnotation, final Object instance, final Object... args) throws IllegalAccessException, IllegalArgumentException, InvocationTargetException { + final List> annotationClasses = new ArrayList<>(alternateAnnotation == null ? 1 : 2); + annotationClasses.add(preferredAnnotation); + if ( alternateAnnotation != null ) { + annotationClasses.add(alternateAnnotation); + } + + boolean annotationFound = false; + for ( final Class annotationClass : annotationClasses ) { + if ( annotationFound ) { + break; + } + + try { + for (final Method method : instance.getClass().getMethods()) { + if (method.isAnnotationPresent(annotationClass)) { + annotationFound = true; + final boolean isAccessible = method.isAccessible(); + method.setAccessible(true); + + try { + final Class[] argumentTypes = method.getParameterTypes(); + if (argumentTypes.length > args.length) { + throw new IllegalArgumentException(String.format("Unable to invoke method %1$s on %2$s because method expects %3$s parameters but only %4$s were given", + method.getName(), instance, argumentTypes.length, args.length)); } - } - - if (argumentTypes.length == args.length) { - method.invoke(instance, args); - } else { - final Object[] argsToPass = new Object[argumentTypes.length]; - for (int i = 0; i < argsToPass.length; i++) { - argsToPass[i] = args[i]; + + for (int i = 0; i < argumentTypes.length; i++) { + final Class argType = argumentTypes[i]; + if (!argType.isAssignableFrom(args[i].getClass())) { + throw new IllegalArgumentException(String.format( + "Unable to invoke method %1$s on %2$s because method parameter %3$s is expected to be of type %4$s but argument passed was of type %5$s", + method.getName(), instance, i, argType, args[i].getClass())); + } + } + + if (argumentTypes.length == args.length) { + method.invoke(instance, args); + } else { + final Object[] argsToPass = new Object[argumentTypes.length]; + for (int i = 0; i < argsToPass.length; i++) { + argsToPass[i] = args[i]; + } + + method.invoke(instance, argsToPass); + } + } finally { + if (!isAccessible) { + method.setAccessible(false); } - - method.invoke(instance, argsToPass); - } - } finally { - if (!isAccessible) { - method.setAccessible(false); } } } - } - } catch (final InvocationTargetException ite) { - if ( ite.getCause() instanceof RuntimeException ) { - throw (RuntimeException) ite.getCause(); - } else { - throw ite; + } catch (final InvocationTargetException ite) { + if ( ite.getCause() instanceof RuntimeException ) { + throw (RuntimeException) ite.getCause(); + } else { + throw ite; + } } } } + /** * Invokes all methods on the given instance that have been annotated with * the given Annotation. If the signature of the method that is defined in @@ -107,47 +148,86 @@ public class ReflectionUtils { * is returned, an error will have been logged. */ public static boolean quietlyInvokeMethodsWithAnnotation(final Class annotation, final Object instance, final Object... args) { - for (final Method method : instance.getClass().getMethods()) { - if (method.isAnnotationPresent(annotation)) { - final boolean isAccessible = method.isAccessible(); - method.setAccessible(true); - - try { - final Class[] argumentTypes = method.getParameterTypes(); - if (argumentTypes.length > args.length) { - LOG.error("Unable to invoke method {} on {} because method expects {} parameters but only {} were given", - new Object[]{method.getName(), instance, argumentTypes.length, args.length}); - return false; - } - - for (int i = 0; i < argumentTypes.length; i++) { - final Class argType = argumentTypes[i]; - if (!argType.isAssignableFrom(args[i].getClass())) { - LOG.error("Unable to invoke method {} on {} because method parameter {} is expected to be of type {} but argument passed was of type {}", - new Object[]{method.getName(), instance, i, argType, args[i].getClass()}); + return quietlyInvokeMethodsWithAnnotation(annotation, null, instance, args); + } + + + /** + * Invokes all methods on the given instance that have been annotated with + * the given preferredAnnotation and if no such method exists will invoke all methods + * on the given instance that have been annotated with the given + * alternateAnnotation, if any exists. If the signature of the method that is defined in + * instance uses 1 or more parameters, those parameters must be + * specified by the args parameter. However, if more arguments + * are supplied by the args parameter than needed, the extra + * arguments will be ignored. + * + * @param preferredAnnotation + * @param alternateAnnotation + * @param instance + * @param args + * @return true if all appropriate methods were invoked and + * returned without throwing an Exception, false if one of the + * methods threw an Exception or could not be invoked; if false + * is returned, an error will have been logged. + */ + public static boolean quietlyInvokeMethodsWithAnnotation(final Class preferredAnnotation, final Class alternateAnnotation, final Object instance, final Object... args) { + final List> annotationClasses = new ArrayList<>(alternateAnnotation == null ? 1 : 2); + annotationClasses.add(preferredAnnotation); + if ( alternateAnnotation != null ) { + annotationClasses.add(alternateAnnotation); + } + + boolean annotationFound = false; + for ( final Class annotationClass : annotationClasses ) { + if ( annotationFound ) { + break; + } + + for (final Method method : instance.getClass().getMethods()) { + if (method.isAnnotationPresent(annotationClass)) { + annotationFound = true; + + final boolean isAccessible = method.isAccessible(); + method.setAccessible(true); + + try { + final Class[] argumentTypes = method.getParameterTypes(); + if (argumentTypes.length > args.length) { + LOG.error("Unable to invoke method {} on {} because method expects {} parameters but only {} were given", + new Object[]{method.getName(), instance, argumentTypes.length, args.length}); return false; } - } - - try { - if (argumentTypes.length == args.length) { - method.invoke(instance, args); - } else { - final Object[] argsToPass = new Object[argumentTypes.length]; - for (int i = 0; i < argsToPass.length; i++) { - argsToPass[i] = args[i]; + + for (int i = 0; i < argumentTypes.length; i++) { + final Class argType = argumentTypes[i]; + if (!argType.isAssignableFrom(args[i].getClass())) { + LOG.error("Unable to invoke method {} on {} because method parameter {} is expected to be of type {} but argument passed was of type {}", + new Object[]{method.getName(), instance, i, argType, args[i].getClass()}); + return false; } - - method.invoke(instance, argsToPass); } - } catch (final IllegalAccessException | IllegalArgumentException | InvocationTargetException t) { - LOG.error("Unable to invoke method {} on {} due to {}", new Object[]{method.getName(), instance, t}); - LOG.error("", t); - return false; - } - } finally { - if (!isAccessible) { - method.setAccessible(false); + + try { + if (argumentTypes.length == args.length) { + method.invoke(instance, args); + } else { + final Object[] argsToPass = new Object[argumentTypes.length]; + for (int i = 0; i < argsToPass.length; i++) { + argsToPass[i] = args[i]; + } + + method.invoke(instance, argsToPass); + } + } catch (final IllegalAccessException | IllegalArgumentException | InvocationTargetException t) { + LOG.error("Unable to invoke method {} on {} due to {}", new Object[]{method.getName(), instance, t}); + LOG.error("", t); + return false; + } + } finally { + if (!isAccessible) { + method.setAccessible(false); + } } } } diff --git a/nifi/nar-bundles/framework-bundle/framework/core/src/test/java/org/apache/nifi/test/processors/StubAttributeLoggerProcessor.java b/nifi/nar-bundles/framework-bundle/framework/core/src/test/java/org/apache/nifi/test/processors/StubAttributeLoggerProcessor.java index 73d38e8c77..d49db29451 100644 --- a/nifi/nar-bundles/framework-bundle/framework/core/src/test/java/org/apache/nifi/test/processors/StubAttributeLoggerProcessor.java +++ b/nifi/nar-bundles/framework-bundle/framework/core/src/test/java/org/apache/nifi/test/processors/StubAttributeLoggerProcessor.java @@ -22,13 +22,13 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.SideEffectFree; @SideEffectFree public class StubAttributeLoggerProcessor extends AbstractProcessor { diff --git a/nifi/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java b/nifi/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java index 9a2dc30adb..92a54499d1 100644 --- a/nifi/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java +++ b/nifi/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java @@ -46,6 +46,8 @@ import org.apache.nifi.action.details.ConfigureDetails; import org.apache.nifi.action.details.ConnectDetails; import org.apache.nifi.action.details.MoveDetails; import org.apache.nifi.action.details.PurgeDetails; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.authorization.Authority; import org.apache.nifi.cluster.HeartbeatPayload; import org.apache.nifi.cluster.event.Event; @@ -82,8 +84,6 @@ import org.apache.nifi.groups.RemoteProcessGroup; import org.apache.nifi.history.History; import org.apache.nifi.processor.Processor; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.Tags; import org.apache.nifi.provenance.lineage.ComputeLineageResult; import org.apache.nifi.provenance.lineage.ComputeLineageSubmission; import org.apache.nifi.provenance.lineage.LineageEdge; @@ -140,7 +140,6 @@ public final class DtoFactory { final int MAX_BULLETINS_PER_COMPONENT = 5; - private NiFiProperties properties; private ControllerServiceLookup controllerServiceLookup; /** @@ -1083,9 +1082,17 @@ public final class DtoFactory { * @param cls * @return */ + @SuppressWarnings("deprecation") private String getCapabilityDescription(final Class cls) { - final CapabilityDescription description = cls.getAnnotation(CapabilityDescription.class); - return (description == null) ? null : description.value(); + final CapabilityDescription capabilityDesc = cls.getAnnotation(CapabilityDescription.class); + if ( capabilityDesc != null ) { + return capabilityDesc.value(); + } + + final org.apache.nifi.processor.annotation.CapabilityDescription deprecatedCapabilityDesc = + cls.getAnnotation(org.apache.nifi.processor.annotation.CapabilityDescription.class); + + return (deprecatedCapabilityDesc == null) ? null : deprecatedCapabilityDesc.value(); } /** @@ -1094,6 +1101,7 @@ public final class DtoFactory { * @param cls * @return */ + @SuppressWarnings("deprecation") private Set getTags(final Class cls) { final Set tags = new HashSet<>(); final Tags tagsAnnotation = cls.getAnnotation(Tags.class); @@ -1101,6 +1109,13 @@ public final class DtoFactory { for (final String tag : tagsAnnotation.value()) { tags.add(tag); } + } else { + final org.apache.nifi.processor.annotation.Tags deprecatedTagsAnnotation = cls.getAnnotation(org.apache.nifi.processor.annotation.Tags.class); + if ( deprecatedTagsAnnotation != null ) { + for ( final String tag : deprecatedTagsAnnotation.value() ) { + tags.add(tag); + } + } } return tags; @@ -2132,7 +2147,6 @@ public final class DtoFactory { /* setters */ public void setProperties(NiFiProperties properties) { - this.properties = properties; } public void setControllerServiceLookup(ControllerServiceLookup lookup) { diff --git a/nifi/nar-bundles/hadoop-bundle/hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java b/nifi/nar-bundles/hadoop-bundle/hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java index dde752b36a..8d5749b039 100644 --- a/nifi/nar-bundles/hadoop-bundle/hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java +++ b/nifi/nar-bundles/hadoop-bundle/hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java @@ -28,6 +28,8 @@ import java.util.concurrent.atomic.AtomicReference; import javax.net.SocketFactory; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; @@ -35,10 +37,7 @@ import org.apache.nifi.components.Validator; import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessorInitializationContext; -import org.apache.nifi.processor.annotation.OnScheduled; -import org.apache.nifi.processor.annotation.OnStopped; import org.apache.nifi.util.Tuple; - import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; diff --git a/nifi/nar-bundles/hadoop-bundle/hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/CreateHadoopSequenceFile.java b/nifi/nar-bundles/hadoop-bundle/hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/CreateHadoopSequenceFile.java index 28e9d8d6a0..1422a7b599 100644 --- a/nifi/nar-bundles/hadoop-bundle/hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/CreateHadoopSequenceFile.java +++ b/nifi/nar-bundles/hadoop-bundle/hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/CreateHadoopSequenceFile.java @@ -22,18 +22,17 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processors.hadoop.util.SequenceFileWriter; - import org.apache.hadoop.io.SequenceFile.CompressionType; /** diff --git a/nifi/nar-bundles/hadoop-bundle/hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java b/nifi/nar-bundles/hadoop-bundle/hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java index 20ac7381fb..0610d8f313 100644 --- a/nifi/nar-bundles/hadoop-bundle/hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java +++ b/nifi/nar-bundles/hadoop-bundle/hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java @@ -31,6 +31,10 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.regex.Pattern; +import org.apache.nifi.annotation.behavior.TriggerWhenEmpty; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; @@ -40,14 +44,9 @@ import org.apache.nifi.processor.DataUnit; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.OnScheduled; -import org.apache.nifi.processor.annotation.Tags; -import org.apache.nifi.processor.annotation.TriggerWhenEmpty; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.util.StopWatch; - import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; diff --git a/nifi/nar-bundles/hadoop-bundle/hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFSSequenceFile.java b/nifi/nar-bundles/hadoop-bundle/hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFSSequenceFile.java index 5581430e8e..ec8b5e67d4 100644 --- a/nifi/nar-bundles/hadoop-bundle/hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFSSequenceFile.java +++ b/nifi/nar-bundles/hadoop-bundle/hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFSSequenceFile.java @@ -22,19 +22,18 @@ import java.util.List; import java.util.Set; import java.util.concurrent.TimeUnit; +import org.apache.nifi.annotation.behavior.TriggerWhenEmpty; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ProcessorLog; import org.apache.nifi.processor.DataUnit; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.Tags; -import org.apache.nifi.processor.annotation.TriggerWhenEmpty; import org.apache.nifi.processors.hadoop.util.SequenceFileReader; import org.apache.nifi.util.StopWatch; import org.apache.nifi.util.Tuple; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; diff --git a/nifi/nar-bundles/hadoop-bundle/hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java b/nifi/nar-bundles/hadoop-bundle/hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java index e84b575416..9a5aa7475b 100644 --- a/nifi/nar-bundles/hadoop-bundle/hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java +++ b/nifi/nar-bundles/hadoop-bundle/hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java @@ -26,6 +26,9 @@ import java.util.List; import java.util.Set; import java.util.concurrent.TimeUnit; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyValue; import org.apache.nifi.components.ValidationContext; @@ -39,9 +42,6 @@ import org.apache.nifi.processor.DataUnit; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.OnScheduled; -import org.apache.nifi.processor.annotation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; diff --git a/nifi/nar-bundles/kafka-bundle/kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java b/nifi/nar-bundles/kafka-bundle/kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java index f202e29afe..5383e9ded5 100644 --- a/nifi/nar-bundles/kafka-bundle/kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java +++ b/nifi/nar-bundles/kafka-bundle/kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java @@ -39,6 +39,12 @@ import kafka.consumer.KafkaStream; import kafka.javaapi.consumer.ConsumerConnector; import kafka.message.MessageAndMetadata; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.annotation.lifecycle.OnUnscheduled; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.Validator; import org.apache.nifi.flowfile.FlowFile; @@ -46,12 +52,6 @@ import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.OnScheduled; -import org.apache.nifi.processor.annotation.OnStopped; -import org.apache.nifi.processor.annotation.OnUnscheduled; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; diff --git a/nifi/nar-bundles/kafka-bundle/kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java b/nifi/nar-bundles/kafka-bundle/kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java index 51f9ef14a2..4df4e08264 100644 --- a/nifi/nar-bundles/kafka-bundle/kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java +++ b/nifi/nar-bundles/kafka-bundle/kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java @@ -33,6 +33,10 @@ import kafka.javaapi.producer.Producer; import kafka.producer.KeyedMessage; import kafka.producer.ProducerConfig; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; @@ -41,10 +45,6 @@ import org.apache.nifi.processor.DataUnit; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.OnStopped; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; diff --git a/nifi/nar-bundles/kafka-bundle/kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestPutKafka.java b/nifi/nar-bundles/kafka-bundle/kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestPutKafka.java index 06a4604efa..b0f239451a 100644 --- a/nifi/nar-bundles/kafka-bundle/kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestPutKafka.java +++ b/nifi/nar-bundles/kafka-bundle/kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestPutKafka.java @@ -31,8 +31,8 @@ import kafka.javaapi.producer.Producer; import kafka.producer.KeyedMessage; import kafka.producer.ProducerConfig; +import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.processor.ProcessContext; -import org.apache.nifi.processor.annotation.OnScheduled; import org.apache.nifi.util.MockFlowFile; import org.apache.nifi.util.TestRunner; import org.apache.nifi.util.TestRunners; diff --git a/nifi/nar-bundles/monitor-threshold-bundle/processor/src/main/java/org/apache/nifi/processors/monitor/MonitorThreshold.java b/nifi/nar-bundles/monitor-threshold-bundle/processor/src/main/java/org/apache/nifi/processors/monitor/MonitorThreshold.java index ec071083a0..1cffc05b3b 100644 --- a/nifi/nar-bundles/monitor-threshold-bundle/processor/src/main/java/org/apache/nifi/processors/monitor/MonitorThreshold.java +++ b/nifi/nar-bundles/monitor-threshold-bundle/processor/src/main/java/org/apache/nifi/processors/monitor/MonitorThreshold.java @@ -39,6 +39,10 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.regex.Pattern; +import org.apache.nifi.annotation.behavior.TriggerWhenEmpty; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; @@ -50,10 +54,6 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.OnStopped; -import org.apache.nifi.processor.annotation.Tags; -import org.apache.nifi.processor.annotation.TriggerWhenEmpty; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.util.Tuple; diff --git a/nifi/nar-bundles/standard-bundle/jms-processors/src/main/java/org/apache/nifi/processors/jms/GetJMSQueue.java b/nifi/nar-bundles/standard-bundle/jms-processors/src/main/java/org/apache/nifi/processors/jms/GetJMSQueue.java index 4c53d0edee..dd48577bc6 100644 --- a/nifi/nar-bundles/standard-bundle/jms-processors/src/main/java/org/apache/nifi/processors/jms/GetJMSQueue.java +++ b/nifi/nar-bundles/standard-bundle/jms-processors/src/main/java/org/apache/nifi/processors/jms/GetJMSQueue.java @@ -21,13 +21,13 @@ import java.util.concurrent.LinkedBlockingQueue; import javax.jms.JMSException; +import org.apache.nifi.annotation.behavior.TriggerWhenEmpty; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.logging.ProcessorLog; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.OnStopped; -import org.apache.nifi.processor.annotation.Tags; -import org.apache.nifi.processor.annotation.TriggerWhenEmpty; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processors.jms.util.JmsFactory; import org.apache.nifi.processors.jms.util.WrappedMessageConsumer; diff --git a/nifi/nar-bundles/standard-bundle/jms-processors/src/main/java/org/apache/nifi/processors/jms/GetJMSTopic.java b/nifi/nar-bundles/standard-bundle/jms-processors/src/main/java/org/apache/nifi/processors/jms/GetJMSTopic.java index 904bd20092..820b1468f3 100644 --- a/nifi/nar-bundles/standard-bundle/jms-processors/src/main/java/org/apache/nifi/processors/jms/GetJMSTopic.java +++ b/nifi/nar-bundles/standard-bundle/jms-processors/src/main/java/org/apache/nifi/processors/jms/GetJMSTopic.java @@ -41,17 +41,17 @@ import javax.jms.InvalidDestinationException; import javax.jms.JMSException; import javax.jms.Session; +import org.apache.nifi.annotation.behavior.TriggerSerially; +import org.apache.nifi.annotation.behavior.TriggerWhenEmpty; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnRemoved; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.logging.ProcessorLog; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.OnRemoved; -import org.apache.nifi.processor.annotation.OnScheduled; -import org.apache.nifi.processor.annotation.OnStopped; -import org.apache.nifi.processor.annotation.Tags; -import org.apache.nifi.processor.annotation.TriggerSerially; -import org.apache.nifi.processor.annotation.TriggerWhenEmpty; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processors.jms.util.JmsFactory; import org.apache.nifi.processors.jms.util.JmsProperties; diff --git a/nifi/nar-bundles/standard-bundle/jms-processors/src/main/java/org/apache/nifi/processors/jms/PutJMS.java b/nifi/nar-bundles/standard-bundle/jms-processors/src/main/java/org/apache/nifi/processors/jms/PutJMS.java index 062423897b..a76c1a2e4b 100644 --- a/nifi/nar-bundles/standard-bundle/jms-processors/src/main/java/org/apache/nifi/processors/jms/PutJMS.java +++ b/nifi/nar-bundles/standard-bundle/jms-processors/src/main/java/org/apache/nifi/processors/jms/PutJMS.java @@ -69,6 +69,9 @@ import javax.jms.MessageProducer; import javax.jms.Session; import javax.jms.StreamMessage; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.stream.io.StreamUtils; @@ -78,9 +81,6 @@ import org.apache.nifi.processor.DataUnit; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.OnStopped; -import org.apache.nifi.processor.annotation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processors.jms.util.JmsFactory; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/Base64EncodeContent.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/Base64EncodeContent.java index 521bd94689..d9175e06ca 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/Base64EncodeContent.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/Base64EncodeContent.java @@ -16,32 +16,35 @@ */ package org.apache.nifi.processors.standard; -import org.apache.nifi.processor.ProcessContext; -import org.apache.nifi.processor.AbstractProcessor; -import org.apache.nifi.processor.ProcessorInitializationContext; -import org.apache.nifi.processor.ProcessSession; -import org.apache.nifi.processor.Relationship; -import org.apache.nifi.components.PropertyDescriptor; -import org.apache.nifi.flowfile.FlowFile; -import org.apache.nifi.logging.ProcessorLog; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.EventDriven; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; -import org.apache.nifi.processor.io.StreamCallback; -import org.apache.nifi.processors.standard.util.ValidatingBase64InputStream; -import org.apache.nifi.util.StopWatch; - -import org.apache.commons.codec.binary.Base64InputStream; -import org.apache.commons.codec.binary.Base64OutputStream; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; -import java.util.*; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; import java.util.concurrent.TimeUnit; +import org.apache.commons.codec.binary.Base64InputStream; +import org.apache.commons.codec.binary.Base64OutputStream; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ProcessorLog; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.ProcessorInitializationContext; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.processor.io.StreamCallback; +import org.apache.nifi.processors.standard.util.ValidatingBase64InputStream; +import org.apache.nifi.util.StopWatch; + @EventDriven @SideEffectFree @SupportsBatching diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java index 21dfe9364e..cf20f16d40 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java @@ -44,11 +44,11 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.EventDriven; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.io.StreamCallback; import org.apache.nifi.util.ObjectHolder; import org.apache.nifi.util.StopWatch; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ControlRate.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ControlRate.java index a96544245b..2c1a8646ea 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ControlRate.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ControlRate.java @@ -43,10 +43,10 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.Tags; -import org.apache.nifi.processor.annotation.TriggerSerially; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.behavior.TriggerSerially; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.util.timebuffer.EntityAccess; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertCharacterSet.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertCharacterSet.java index d67179f451..119a3f2449 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertCharacterSet.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertCharacterSet.java @@ -24,11 +24,11 @@ import org.apache.nifi.processor.Relationship; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ProcessorLog; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.EventDriven; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.StreamCallback; import org.apache.nifi.processor.util.StandardValidators; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/DetectDuplicate.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/DetectDuplicate.java index 1fea36d2e2..8910fdcd31 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/DetectDuplicate.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/DetectDuplicate.java @@ -39,10 +39,10 @@ import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.EventDriven; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.util.StandardValidators; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java index 3ac55d250b..8fb8ad485d 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java @@ -42,13 +42,13 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.EventDriven; -import org.apache.nifi.processor.annotation.OnScheduled; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; -import org.apache.nifi.processor.annotation.TriggerWhenAnyDestinationAvailable; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.behavior.TriggerWhenAnyDestinationAvailable; import org.apache.nifi.processor.util.StandardValidators; import org.apache.commons.lang3.StringUtils; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/EncryptContent.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/EncryptContent.java index eb079bba06..69cb18e51e 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/EncryptContent.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/EncryptContent.java @@ -25,11 +25,11 @@ import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.stream.io.StreamUtils; import org.apache.nifi.logging.ProcessorLog; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.EventDriven; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.StreamCallback; import org.apache.nifi.processor.util.StandardValidators; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateRegularExpression.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateRegularExpression.java index 7697d06657..4140943547 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateRegularExpression.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateRegularExpression.java @@ -39,11 +39,11 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.EventDriven; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java index a1fc86d000..4827ee339f 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java @@ -65,12 +65,12 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.EventDriven; -import org.apache.nifi.processor.annotation.OnScheduled; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.io.OutputStreamCallback; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java index 8b4ce09bc6..3ddee83d81 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java @@ -62,11 +62,11 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.EventDriven; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.io.OutputStreamCallback; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteStreamCommand.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteStreamCommand.java index ab0b2aa262..dda3647bbb 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteStreamCommand.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteStreamCommand.java @@ -46,10 +46,10 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.EventDriven; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.io.OutputStreamCallback; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateFlowFile.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateFlowFile.java index 6f18a019b2..0ae474733e 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateFlowFile.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateFlowFile.java @@ -34,10 +34,10 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.OnScheduled; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFTP.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFTP.java index 854ee37ad5..2dabbc6c9a 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFTP.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFTP.java @@ -23,9 +23,9 @@ import java.util.List; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessorInitializationContext; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processors.standard.util.FTPTransfer; import org.apache.nifi.processors.standard.util.FileTransfer; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFile.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFile.java index 2152d2ec6b..2a2504c654 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFile.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFile.java @@ -59,10 +59,10 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.OnScheduled; -import org.apache.nifi.processor.annotation.Tags; -import org.apache.nifi.processor.annotation.TriggerWhenEmpty; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.behavior.TriggerWhenEmpty; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.util.StandardValidators; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFileTransfer.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFileTransfer.java index 3edebe832e..1b2be2679d 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFileTransfer.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFileTransfer.java @@ -48,7 +48,7 @@ import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.OnScheduled; +import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.processor.exception.FlowFileAccessException; import org.apache.nifi.processors.standard.util.FileInfo; import org.apache.nifi.processors.standard.util.FileTransfer; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java index 35873b146b..fd70024a82 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java @@ -79,9 +79,9 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessSessionFactory; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.OnShutdown; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.lifecycle.OnShutdown; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.ssl.SSLContextService; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/GetSFTP.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/GetSFTP.java index 077b32fb77..dd9f5199c8 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/GetSFTP.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/GetSFTP.java @@ -26,9 +26,9 @@ import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessorInitializationContext; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processors.standard.util.FileTransfer; import org.apache.nifi.processors.standard.util.SFTPTransfer; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/HashAttribute.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/HashAttribute.java index 5fc7e5a592..e39ab26bf2 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/HashAttribute.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/HashAttribute.java @@ -37,11 +37,11 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.EventDriven; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.util.StandardValidators; import org.apache.commons.codec.digest.DigestUtils; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/HashContent.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/HashContent.java index bb715891f2..827653bec0 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/HashContent.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/HashContent.java @@ -38,10 +38,10 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.EventDriven; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/IdentifyMimeType.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/IdentifyMimeType.java index 50bdba23af..a52ac3b120 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/IdentifyMimeType.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/IdentifyMimeType.java @@ -35,11 +35,11 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.EventDriven; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.util.FlowFilePackagerV1; import org.apache.nifi.util.FlowFilePackagerV3; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java index e5c1ed85e8..41ecd91378 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java @@ -55,9 +55,9 @@ import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.ssl.SSLContextService; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java index b7fe97a80a..d9317c4ce1 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java @@ -42,10 +42,10 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessSessionFactory; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.OnScheduled; -import org.apache.nifi.processor.annotation.OnStopped; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.processors.standard.servlets.ContentAcknowledgmentServlet; import org.apache.nifi.processors.standard.servlets.ListenHTTPServlet; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDP.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDP.java index 43d839552c..561e333f91 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDP.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDP.java @@ -58,12 +58,12 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessSessionFactory; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.OnScheduled; -import org.apache.nifi.processor.annotation.OnStopped; -import org.apache.nifi.processor.annotation.OnUnscheduled; -import org.apache.nifi.processor.annotation.Tags; -import org.apache.nifi.processor.annotation.TriggerWhenEmpty; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.annotation.lifecycle.OnUnscheduled; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.behavior.TriggerWhenEmpty; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.processors.standard.util.UDPStreamConsumer; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/LogAttribute.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/LogAttribute.java index 9708c49b9c..daf513bc5e 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/LogAttribute.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/LogAttribute.java @@ -35,10 +35,10 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.EventDriven; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java index 9311ee9673..f2e4a8dc6a 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java @@ -58,12 +58,12 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessSessionFactory; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.OnScheduled; -import org.apache.nifi.processor.annotation.OnStopped; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.Tags; -import org.apache.nifi.processor.annotation.TriggerWhenEmpty; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.behavior.TriggerWhenEmpty; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.io.OutputStreamCallback; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyBytes.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyBytes.java index a8b190d1ef..ccebb46ff0 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyBytes.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyBytes.java @@ -34,10 +34,10 @@ import org.apache.nifi.processor.DataUnit; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.EventDriven; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.processor.io.StreamCallback; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/MonitorActivity.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/MonitorActivity.java index 115f234cd6..c5fce3c6bb 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/MonitorActivity.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/MonitorActivity.java @@ -39,11 +39,11 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.Tags; -import org.apache.nifi.processor.annotation.TriggerSerially; -import org.apache.nifi.processor.annotation.TriggerWhenEmpty; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.behavior.TriggerSerially; +import org.apache.nifi.annotation.behavior.TriggerWhenEmpty; import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java index ee16610e37..adea255a79 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java @@ -91,11 +91,11 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.OnScheduled; -import org.apache.nifi.processor.annotation.OnStopped; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutEmail.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutEmail.java index ef602c9375..2fa71c8994 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutEmail.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutEmail.java @@ -53,9 +53,9 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFTP.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFTP.java index 6c8a816df9..dac367ffbe 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFTP.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFTP.java @@ -30,10 +30,10 @@ import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessorInitializationContext; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.OnScheduled; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.processors.standard.util.FTPTransfer; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFile.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFile.java index 6db275796d..70ac5acb1a 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFile.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFile.java @@ -43,9 +43,9 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.util.StopWatch; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSFTP.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSFTP.java index a8d9c18f29..cfd522c703 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSFTP.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSFTP.java @@ -23,9 +23,9 @@ import java.util.List; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessorInitializationContext; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processors.standard.util.SFTPTransfer; @SupportsBatching diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java index ae5350b50b..111dead352 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java @@ -30,11 +30,11 @@ import org.apache.nifi.expression.AttributeValueDecorator; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.stream.io.StreamUtils; import org.apache.nifi.logging.ProcessorLog; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.EventDriven; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.processor.io.StreamCallback; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceTextWithMapping.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceTextWithMapping.java index c99935bc1b..c4dd83ae6d 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceTextWithMapping.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceTextWithMapping.java @@ -52,11 +52,11 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.EventDriven; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.StreamCallback; import org.apache.nifi.processor.util.StandardValidators; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java index be4aed6594..1fe78af956 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java @@ -37,11 +37,11 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.EventDriven; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.util.StandardValidators; /** diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java index cb3cff28e7..3e581d2a26 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java @@ -40,11 +40,11 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.EventDriven; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.util.IntegerHolder; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java index df13c667f7..6d48d02dfa 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java @@ -42,12 +42,12 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.EventDriven; -import org.apache.nifi.processor.annotation.OnScheduled; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.util.StandardValidators; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanContent.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanContent.java index 9f534695f7..d9f20349d1 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanContent.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanContent.java @@ -45,11 +45,11 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.EventDriven; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/SegmentContent.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/SegmentContent.java index 1df4de69e2..dfdd401033 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/SegmentContent.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/SegmentContent.java @@ -34,11 +34,11 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.EventDriven; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.util.StandardValidators; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitContent.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitContent.java index 7e67c01f41..8c00a7e650 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitContent.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitContent.java @@ -41,11 +41,11 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.EventDriven; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.util.NaiveSearchRingBuffer; import org.apache.nifi.util.Tuple; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java index 8520a55c7c..2a81b64754 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java @@ -29,11 +29,11 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.EventDriven; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitXml.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitXml.java index c5eda3d81a..1220b7c530 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitXml.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitXml.java @@ -38,11 +38,11 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.EventDriven; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java index 738591814c..5e251f6c1a 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java @@ -46,11 +46,11 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.EventDriven; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.io.StreamCallback; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.util.StopWatch; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/UnpackContent.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/UnpackContent.java index dc6daea3b0..b30e780b72 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/UnpackContent.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/UnpackContent.java @@ -42,11 +42,11 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.EventDriven; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.io.OutputStreamCallback; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateXml.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateXml.java index 8f2001a7ca..4808a59ef4 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateXml.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateXml.java @@ -39,12 +39,12 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.EventDriven; -import org.apache.nifi.processor.annotation.OnScheduled; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.SupportsBatching; -import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.util.BooleanHolder; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/RESTServiceContentModified.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/RESTServiceContentModified.java index 8548c46256..ec3211c0f4 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/RESTServiceContentModified.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/RESTServiceContentModified.java @@ -35,7 +35,7 @@ public class RESTServiceContentModified extends HttpServlet { public static boolean IGNORE_LAST_MODIFIED = false; public RESTServiceContentModified() { - this.ETAG = this.hashCode(); + ETAG = this.hashCode(); } @Override diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestCompressContent.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestCompressContent.java index 1af48d6a13..71c85831d3 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestCompressContent.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestCompressContent.java @@ -16,13 +16,14 @@ */ package org.apache.nifi.processors.standard; +import static org.junit.Assert.assertTrue; + import java.io.IOException; -import java.nio.file.Files; import java.nio.file.Paths; + import org.apache.nifi.util.MockFlowFile; import org.apache.nifi.util.TestRunner; import org.apache.nifi.util.TestRunners; -import static org.junit.Assert.assertTrue; import org.junit.Test; public class TestCompressContent { diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestConvertCharacterSet.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestConvertCharacterSet.java index 6092761d8b..1b057d901f 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestConvertCharacterSet.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestConvertCharacterSet.java @@ -16,16 +16,13 @@ */ package org.apache.nifi.processors.standard; -import org.apache.nifi.processors.standard.ConvertCharacterSet; import java.io.File; import java.io.IOException; -import java.nio.file.Files; import java.nio.file.Paths; import org.apache.nifi.util.MockFlowFile; import org.apache.nifi.util.TestRunner; import org.apache.nifi.util.TestRunners; - import org.junit.Test; public class TestConvertCharacterSet { diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDetectDuplicate.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDetectDuplicate.java index 4fae6edf55..6296f7db27 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDetectDuplicate.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDetectDuplicate.java @@ -16,7 +16,6 @@ */ package org.apache.nifi.processors.standard; -import org.apache.nifi.processors.standard.DetectDuplicate; import java.io.File; import java.io.IOException; import java.io.OutputStream; @@ -26,6 +25,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import org.apache.commons.lang3.SerializationException; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.controller.AbstractControllerService; import org.apache.nifi.distributed.cache.client.Deserializer; @@ -36,8 +36,6 @@ import org.apache.nifi.reporting.InitializationException; import org.apache.nifi.util.MockControllerServiceInitializationContext; import org.apache.nifi.util.TestRunner; import org.apache.nifi.util.TestRunners; - -import org.apache.commons.lang3.SerializationException; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGetHTTP.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGetHTTP.java index 63bdcf8f55..cf6530a691 100644 --- a/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGetHTTP.java +++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGetHTTP.java @@ -30,7 +30,6 @@ import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.reporting.InitializationException; import org.apache.nifi.ssl.SSLContextService; import org.apache.nifi.ssl.StandardSSLContextService; -import org.apache.nifi.util.MockControllerServiceInitializationContext; import org.apache.nifi.util.MockFlowFile; import org.apache.nifi.util.MockProcessContext; import org.apache.nifi.util.MockProcessorInitializationContext; diff --git a/nifi/nar-bundles/standard-services/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/DistributedCacheServer.java b/nifi/nar-bundles/standard-services/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/DistributedCacheServer.java index 0f962d0249..19d95b3c5b 100644 --- a/nifi/nar-bundles/standard-services/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/DistributedCacheServer.java +++ b/nifi/nar-bundles/standard-services/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/DistributedCacheServer.java @@ -20,11 +20,11 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import org.apache.nifi.annotation.lifecycle.OnShutdown; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.controller.AbstractControllerService; import org.apache.nifi.controller.ConfigurationContext; import org.apache.nifi.controller.annotation.OnConfigured; -import org.apache.nifi.processor.annotation.OnShutdown; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.ssl.SSLContextService; diff --git a/nifi/nar-bundles/update-attribute-bundle/processor/src/main/java/org/apache/nifi/processors/attributes/UpdateAttribute.java b/nifi/nar-bundles/update-attribute-bundle/processor/src/main/java/org/apache/nifi/processors/attributes/UpdateAttribute.java index 3721df92df..5e1e0267d2 100644 --- a/nifi/nar-bundles/update-attribute-bundle/processor/src/main/java/org/apache/nifi/processors/attributes/UpdateAttribute.java +++ b/nifi/nar-bundles/update-attribute-bundle/processor/src/main/java/org/apache/nifi/processors/attributes/UpdateAttribute.java @@ -30,6 +30,11 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicReference; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyValue; import org.apache.nifi.components.ValidationContext; @@ -42,11 +47,6 @@ import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.annotation.CapabilityDescription; -import org.apache.nifi.processor.annotation.EventDriven; -import org.apache.nifi.processor.annotation.OnScheduled; -import org.apache.nifi.processor.annotation.SideEffectFree; -import org.apache.nifi.processor.annotation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.search.SearchContext; @@ -58,7 +58,6 @@ import org.apache.nifi.update.attributes.Criteria; import org.apache.nifi.update.attributes.Rule; import org.apache.nifi.update.attributes.FlowFilePolicy; import org.apache.nifi.update.attributes.serde.CriteriaSerDe; - import org.apache.commons.lang3.StringUtils; /** diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/AbstractSessionFactoryProcessor.java b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/AbstractSessionFactoryProcessor.java index 8ccd14513d..f13a14304d 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/AbstractSessionFactoryProcessor.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/AbstractSessionFactoryProcessor.java @@ -19,11 +19,11 @@ package org.apache.nifi.processor; import java.util.Collections; import java.util.Set; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.lifecycle.OnUnscheduled; import org.apache.nifi.components.AbstractConfigurableComponent; import org.apache.nifi.controller.ControllerServiceLookup; import org.apache.nifi.logging.ProcessorLog; -import org.apache.nifi.processor.annotation.OnScheduled; -import org.apache.nifi.processor.annotation.OnUnscheduled; /** *

diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/CapabilityDescription.java b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/CapabilityDescription.java index c0bdc4d94f..fad1ebb90d 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/CapabilityDescription.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/CapabilityDescription.java @@ -28,11 +28,14 @@ import java.lang.annotation.Target; * provided. This description can be provided to a user in logs, UI, etc. * * @author none + * @deprecated This Annotation has been replaced by the {@link org.apache.nifi.annotation.documentation.CapabilityDescription} + * annotation. */ @Documented @Target({ElementType.TYPE}) @Retention(RetentionPolicy.RUNTIME) @Inherited +@Deprecated public @interface CapabilityDescription { String value(); diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/EventDriven.java b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/EventDriven.java index f4bfe05b89..615216828f 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/EventDriven.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/EventDriven.java @@ -39,11 +39,14 @@ import java.lang.annotation.Target; *

* * @author none + * @deprecated This Annotation has been replaced by the {@link org.apache.nifi.annotation.behavior.EventDriven} + * annotation. */ @Documented @Target({ElementType.TYPE}) @Retention(RetentionPolicy.RUNTIME) @Inherited +@Deprecated public @interface EventDriven { } diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnAdded.java b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnAdded.java index 1d2807e4f2..b2ea5eb43d 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnAdded.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnAdded.java @@ -32,10 +32,12 @@ import java.lang.annotation.Target; * be added to the graph. * * @author none + * @deprecated This Annotation has been replaced by the {@link org.apache.nifi.annotation.lifecycle.OnAdded} annotation. */ @Documented @Target({ElementType.METHOD}) @Retention(RetentionPolicy.RUNTIME) @Inherited +@Deprecated public @interface OnAdded { } diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnRemoved.java b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnRemoved.java index b39709afaf..fae4e34610 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnRemoved.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnRemoved.java @@ -33,10 +33,12 @@ import java.lang.annotation.Target; * be removed from the graph. * * @author none + * @deprecated This Annotation has been replaced by the {@link org.apache.nifi.annotation.lifecycle.OnRemoved} annotation. */ @Documented @Target({ElementType.METHOD}) @Retention(RetentionPolicy.RUNTIME) @Inherited +@Deprecated public @interface OnRemoved { } diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnScheduled.java b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnScheduled.java index 5be942c47e..ed65ce0459 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnScheduled.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnScheduled.java @@ -35,10 +35,12 @@ import java.lang.annotation.Target; * be scheduled to run. * * @author none - */ + * @deprecated This Annotation has been replaced by the {@link org.apache.nifi.annotation.lifecycle.OnScheduled} annotation. +*/ @Documented @Target({ElementType.METHOD}) @Retention(RetentionPolicy.RUNTIME) @Inherited +@Deprecated public @interface OnScheduled { } diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnShutdown.java b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnShutdown.java index 60737ad57f..bb38221c28 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnShutdown.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnShutdown.java @@ -29,10 +29,12 @@ import java.lang.annotation.Target; * most once for each processor instance in a process lifetime. * * @author none + * @deprecated This Annotation has been replaced by the {@link org.apache.nifi.annotation.lifecycle.OnShutdown} annotation. */ @Documented @Target({ElementType.METHOD}) @Retention(RetentionPolicy.RUNTIME) @Inherited +@Deprecated public @interface OnShutdown { } diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnStopped.java b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnStopped.java index e6d892952f..3f61850a54 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnStopped.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnStopped.java @@ -46,11 +46,13 @@ import java.lang.annotation.Target; *

* * @author none + * @deprecated This Annotation has been replaced by the {@link org.apache.nifi.annotation.lifecycle.OnStopped} annotation. */ @Documented @Target({ElementType.METHOD}) @Retention(RetentionPolicy.RUNTIME) @Inherited +@Deprecated public @interface OnStopped { } diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnUnscheduled.java b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnUnscheduled.java index 3e80c9392d..a9b94fc17d 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnUnscheduled.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnUnscheduled.java @@ -37,10 +37,12 @@ import java.lang.annotation.Target; * be scheduled to run. * * @author none + * @deprecated This Annotation has been replaced by the {@link org.apache.nifi.annotation.lifecycle.OnUnscheduled} annotation. */ @Documented @Target({ElementType.METHOD}) @Retention(RetentionPolicy.RUNTIME) @Inherited +@Deprecated public @interface OnUnscheduled { } diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/SideEffectFree.java b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/SideEffectFree.java index a27b0f38fc..281b38d902 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/SideEffectFree.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/SideEffectFree.java @@ -36,10 +36,12 @@ import java.lang.annotation.Target; * repeated (implied idempotent behavior). * * @author none + * @deprecated This Annotation has been replaced by the {@link org.apache.nifi.annotation.behavior.SideEffectFree} annotation. */ @Documented @Target({ElementType.TYPE}) @Retention(RetentionPolicy.RUNTIME) @Inherited +@Deprecated public @interface SideEffectFree { } diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/SupportsBatching.java b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/SupportsBatching.java index 016776c04b..2b89e4e9ef 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/SupportsBatching.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/SupportsBatching.java @@ -41,11 +41,13 @@ import java.lang.annotation.Target; * from a remote source. * * @author none + * @deprecated This Annotation has been replaced by the {@link org.apache.nifi.annotation.behavior.SupportsBatching} annotation. */ @Documented @Target({ElementType.TYPE}) @Retention(RetentionPolicy.RUNTIME) @Inherited +@Deprecated public @interface SupportsBatching { } diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/Tags.java b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/Tags.java index 3b8d9c2e92..c06302d52e 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/Tags.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/Tags.java @@ -30,11 +30,14 @@ import java.lang.annotation.Target; * Processors. * * @author none + * @deprecated This Annotation has been replaced by the {@link org.apache.nifi.annotation.documentation.Tags} + * annotation. */ @Documented @Target({ElementType.TYPE}) @Retention(RetentionPolicy.RUNTIME) @Inherited +@Deprecated public @interface Tags { /** diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/TriggerSerially.java b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/TriggerSerially.java index bd9b1b1fec..0b3d1e6d05 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/TriggerSerially.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/TriggerSerially.java @@ -30,10 +30,12 @@ import java.lang.annotation.Target; * execution. * * @author none - */ + * @deprecated This Annotation has been replaced by the {@link org.apache.nifi.annotation.behavior.TriggerSerially} annotation. +*/ @Documented @Target({ElementType.TYPE}) @Retention(RetentionPolicy.RUNTIME) @Inherited +@Deprecated public @interface TriggerSerially { } diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/TriggerWhenAnyDestinationAvailable.java b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/TriggerWhenAnyDestinationAvailable.java index 13d48f8510..52f6c5e132 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/TriggerWhenAnyDestinationAvailable.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/TriggerWhenAnyDestinationAvailable.java @@ -30,11 +30,13 @@ import java.lang.annotation.Target; * destinations report that they have available space. * * @author none + * @deprecated This Annotation has been replaced by the {@link org.apache.nifi.annotation.behavior.TriggerWhenAnyDestinationAvailable} annotation. */ @Documented @Target({ElementType.TYPE}) @Retention(RetentionPolicy.RUNTIME) @Inherited +@Deprecated public @interface TriggerWhenAnyDestinationAvailable { } diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/TriggerWhenEmpty.java b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/TriggerWhenEmpty.java index d3093320e7..1d2f755f48 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/TriggerWhenEmpty.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/annotation/TriggerWhenEmpty.java @@ -32,10 +32,12 @@ import java.lang.annotation.Target; * their queue or they present this annotation. * * @author none + * @deprecated This Annotation has been replaced by the {@link org.apache.nifi.annotation.behavior.TriggerWhenEmpty} annotation. */ @Documented @Target({ElementType.TYPE}) @Retention(RetentionPolicy.RUNTIME) @Inherited +@Deprecated public @interface TriggerWhenEmpty { }