From 3f4c3bd1ccc544edac9b6f4937519bfdab32ba44 Mon Sep 17 00:00:00 2001 From: joewitt Date: Wed, 13 Jul 2016 18:42:10 -0400 Subject: [PATCH 1/3] NIFI-1307 removed deprecation indication for getId and provided better API documentation --- .../main/java/org/apache/nifi/flowfile/FlowFile.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/nifi-api/src/main/java/org/apache/nifi/flowfile/FlowFile.java b/nifi-api/src/main/java/org/apache/nifi/flowfile/FlowFile.java index 43fbf20054c7..9063a8f64c0c 100644 --- a/nifi-api/src/main/java/org/apache/nifi/flowfile/FlowFile.java +++ b/nifi-api/src/main/java/org/apache/nifi/flowfile/FlowFile.java @@ -30,13 +30,13 @@ public interface FlowFile extends Comparable { /** - * @return the unique identifier for this flow file - * @deprecated This method has been deprecated in favor of using the attribute - * {@link org.apache.nifi.flowfile.attributes.CoreAttributes.UUID CoreAttributes.UUID}. - * If an identifier is needed use {@link #getAttribute(String)} to retrieve the value for this attribute. - * For example, by calling getAttribute(CoreAttributes.UUID.getKey()). + * @return the unique identifier for this flow file which is guaranteed + * to be unique within a single running instance of nifi. This identifier + * should not be used for true universal unique type needs. For that consider + * using the attribute found in the flow file's attribute map keyed by + * {@link org.apache.nifi.flowfile.attributes.CoreAttributes.UUID CoreAttributes.UUID}. + * For example, by calling getAttribute(CoreAttributes.UUID.getKey()). */ - @Deprecated long getId(); /** From a2ab663a53150169a754012479df002aa9e43a89 Mon Sep 17 00:00:00 2001 From: joewitt Date: Wed, 13 Jul 2016 19:36:03 -0400 Subject: [PATCH 2/3] NIFI-1157 resolved deprecated nifi-api items and ripple effects --- .../controller/AbstractControllerService.java | 4 +- .../controller/annotation/OnConfigured.java | 44 ----- .../org/apache/nifi/flowfile/FlowFile.java | 20 --- .../annotation/CapabilityDescription.java | 42 ----- .../processor/annotation/EventDriven.java | 51 ------ .../nifi/processor/annotation/OnAdded.java | 43 ----- .../nifi/processor/annotation/OnRemoved.java | 44 ----- .../processor/annotation/OnScheduled.java | 46 ------ .../nifi/processor/annotation/OnShutdown.java | 40 ----- .../nifi/processor/annotation/OnStopped.java | 58 ------- .../processor/annotation/OnUnscheduled.java | 48 ------ .../processor/annotation/SideEffectFree.java | 47 ------ .../annotation/SupportsBatching.java | 53 ------ .../nifi/processor/annotation/Tags.java | 43 ----- .../processor/annotation/TriggerSerially.java | 41 ----- .../TriggerWhenAnyDestinationAvailable.java | 43 ----- .../annotation/TriggerWhenEmpty.java | 43 ----- .../provenance/ProvenanceEventBuilder.java | 10 -- .../provenance/ProvenanceEventRecord.java | 14 -- .../nifi/provenance/lineage/LineageNode.java | 7 - .../apache/nifi/reporting/ReportingTask.java | 4 +- .../PlaceholderProvenanceEvent.java | 5 - .../StandardProvenanceEventRecord.java | 26 +-- .../nifi/provenance/lineage/EventNode.java | 11 -- .../nifi/provenance/lineage/FlowFileNode.java | 7 - .../expression/language/TestQuery.java | 1 - .../org/apache/nifi/util/MockFlowFile.java | 9 -- .../util/StandardProcessorTestRunner.java | 49 ------ .../util/TestStandardProcessorTestRunner.java | 47 ------ .../processors/flume/util/FlowFileEvent.java | 6 - .../flume/util/FlowFileEventConstants.java | 3 - .../init/ControllerServiceInitializer.java | 2 +- .../init/ProcessorInitializer.java | 2 +- .../init/ReportingTaskingInitializer.java | 2 +- .../documentation/util/ReflectionUtils.java | 152 +++++++++--------- .../controller/FileSystemSwapManager.java | 21 +-- .../nifi/controller/FlowController.java | 8 +- .../controller/StandardProcessorNode.java | 33 +--- .../reporting/AbstractReportingTaskNode.java | 25 +-- .../repository/StandardFlowFileRecord.java | 29 ---- .../repository/StandardProcessSession.java | 7 - .../WriteAheadFlowFileRepository.java | 30 ++-- .../EventDrivenSchedulingAgent.java | 2 +- .../StandardControllerServiceNode.java | 21 +-- .../tasks/ContinuallyRunConnectableTask.java | 2 +- .../tasks/ReportingTaskWrapper.java | 4 +- .../nifi/groups/StandardProcessGroup.java | 4 +- .../controller/TestFileSystemSwapManager.java | 7 - .../controller/TestStandardFlowFileQueue.java | 11 +- .../PersistentProvenanceRepository.java | 2 +- .../nifi/provenance/StandardRecordReader.java | 17 +- .../nifi/provenance/StandardRecordWriter.java | 2 - .../provenance/lucene/IndexingAction.java | 4 - .../nifi/provenance/lucene/LineageQuery.java | 25 +-- .../org/apache/nifi/provenance/TestUtil.java | 7 - .../VolatileProvenanceRepository.java | 6 - .../TestVolatileProvenanceRepository.java | 7 - .../SiteToSiteProvenanceReportingTask.java | 10 -- ...TestSiteToSiteProvenanceReportingTask.java | 7 - 59 files changed, 129 insertions(+), 1229 deletions(-) delete mode 100644 nifi-api/src/main/java/org/apache/nifi/controller/annotation/OnConfigured.java delete mode 100644 nifi-api/src/main/java/org/apache/nifi/processor/annotation/CapabilityDescription.java delete mode 100644 nifi-api/src/main/java/org/apache/nifi/processor/annotation/EventDriven.java delete mode 100644 nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnAdded.java delete mode 100644 nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnRemoved.java delete mode 100644 nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnScheduled.java delete mode 100644 nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnShutdown.java delete mode 100644 nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnStopped.java delete mode 100644 nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnUnscheduled.java delete mode 100644 nifi-api/src/main/java/org/apache/nifi/processor/annotation/SideEffectFree.java delete mode 100644 nifi-api/src/main/java/org/apache/nifi/processor/annotation/SupportsBatching.java delete mode 100644 nifi-api/src/main/java/org/apache/nifi/processor/annotation/Tags.java delete mode 100644 nifi-api/src/main/java/org/apache/nifi/processor/annotation/TriggerSerially.java delete mode 100644 nifi-api/src/main/java/org/apache/nifi/processor/annotation/TriggerWhenAnyDestinationAvailable.java delete mode 100644 nifi-api/src/main/java/org/apache/nifi/processor/annotation/TriggerWhenEmpty.java diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/AbstractControllerService.java b/nifi-api/src/main/java/org/apache/nifi/controller/AbstractControllerService.java index 83d3301bfb67..15d30c3caab7 100644 --- a/nifi-api/src/main/java/org/apache/nifi/controller/AbstractControllerService.java +++ b/nifi-api/src/main/java/org/apache/nifi/controller/AbstractControllerService.java @@ -17,12 +17,12 @@ package org.apache.nifi.controller; import java.util.Map; +import org.apache.nifi.annotation.lifecycle.OnConfigurationRestored; import org.apache.nifi.components.AbstractConfigurableComponent; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyValue; import org.apache.nifi.components.state.StateManager; -import org.apache.nifi.controller.annotation.OnConfigured; import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.reporting.InitializationException; @@ -49,7 +49,7 @@ public String getIdentifier() { return identifier; } - @OnConfigured + @OnConfigurationRestored public void onConfigurationChange(final ConfigurationContext context) { this.configContext = context; } diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/annotation/OnConfigured.java b/nifi-api/src/main/java/org/apache/nifi/controller/annotation/OnConfigured.java deleted file mode 100644 index d56b969fb23c..000000000000 --- a/nifi-api/src/main/java/org/apache/nifi/controller/annotation/OnConfigured.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.controller.annotation; - -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 ControllerService implementation can use to indicate a - * method should be called after all of the properties have been set for the - * Controller Service. Methods using this annotation must take either 0 - * arguments or a single argument of type - * {@link org.apache.nifi.controller.ConfigurationContext ConfigurationContext}. - * - * - * @deprecated This annotation has been replaced by those in the - * {@link org.apache.nifi.annotation.lifecycle} package. - */ -@Documented -@Target({ElementType.METHOD}) -@Retention(RetentionPolicy.RUNTIME) -@Inherited -@Deprecated -public @interface OnConfigured { - -} diff --git a/nifi-api/src/main/java/org/apache/nifi/flowfile/FlowFile.java b/nifi-api/src/main/java/org/apache/nifi/flowfile/FlowFile.java index 9063a8f64c0c..0f4d2d015d33 100644 --- a/nifi-api/src/main/java/org/apache/nifi/flowfile/FlowFile.java +++ b/nifi-api/src/main/java/org/apache/nifi/flowfile/FlowFile.java @@ -17,7 +17,6 @@ package org.apache.nifi.flowfile; import java.util.Map; -import java.util.Set; /** *

@@ -86,25 +85,6 @@ public interface FlowFile extends Comparable { */ long getQueueDateIndex(); - /** - *

- * If a FlowFile is derived from multiple "parent" FlowFiles, all of the - * parents' Lineage Identifiers will be in the set. - *

- * - * @return a set of identifiers that are unique to this FlowFile's lineage. - * If FlowFile X is derived from FlowFile Y, both FlowFiles will have the - * same value for the Lineage Claim ID. - * - * @deprecated this collection was erroneously unbounded and caused a lot of OutOfMemoryError problems - * when dealing with FlowFiles with many ancestors. This Collection is - * now capped at 100 lineage identifiers. This method was introduced with the idea of providing - * future performance improvements but due to the high cost of heap consumption will not be used - * in such a manner. As a result, this method will be removed in a future release. - */ - @Deprecated - Set getLineageIdentifiers(); - /** * @return true if flow file is currently penalized; false otherwise; */ diff --git a/nifi-api/src/main/java/org/apache/nifi/processor/annotation/CapabilityDescription.java b/nifi-api/src/main/java/org/apache/nifi/processor/annotation/CapabilityDescription.java deleted file mode 100644 index f966f89d2817..000000000000 --- a/nifi-api/src/main/java/org/apache/nifi/processor/annotation/CapabilityDescription.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * 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.processor.annotation; - -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 allowing for a description to be - * provided. This description can be provided to a user in logs, UI, etc. - * - * @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-api/src/main/java/org/apache/nifi/processor/annotation/EventDriven.java b/nifi-api/src/main/java/org/apache/nifi/processor/annotation/EventDriven.java deleted file mode 100644 index 0f412ca5b2fa..000000000000 --- a/nifi-api/src/main/java/org/apache/nifi/processor/annotation/EventDriven.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * 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.processor.annotation; - -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. - *

- * - * @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-api/src/main/java/org/apache/nifi/processor/annotation/OnAdded.java b/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnAdded.java deleted file mode 100644 index ff0b75fbf894..000000000000 --- a/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnAdded.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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.processor.annotation; - -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 a method - * should be called whenever the processor is added to the graph. This method - * will be called once for the entire life of a processor instance. - * - * If any method annotated with this annotation throws, the processor will not - * be added to the graph. - * - * @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-api/src/main/java/org/apache/nifi/processor/annotation/OnRemoved.java b/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnRemoved.java deleted file mode 100644 index 740e9f860b9f..000000000000 --- a/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnRemoved.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * 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.processor.annotation; - -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 a method - * should be called whenever the processor is removed from the graph. This - * method will be called once for the entire life of a processor instance unless - * an invocation of this method throws any Throwable. - * - * If any method annotated with this annotation throws, the processor will not - * be removed from the graph. - * - * @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-api/src/main/java/org/apache/nifi/processor/annotation/OnScheduled.java b/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnScheduled.java deleted file mode 100644 index 2fec39c2fdcb..000000000000 --- a/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnScheduled.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * 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.processor.annotation; - -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 a method - * should be called whenever the processor is scheduled for processing. This - * will be called before any 'onTrigger' calls and will be called once each time - * a processor instance is scheduled to run. Methods using this annotation must - * take either 0 arguments or a single argument of type - * {@link org.apache.nifi.processor.SchedulingContext SchedulingContext}. - * - * If any method annotated with this annotation throws, the processor will not - * be scheduled to run. - * - * @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-api/src/main/java/org/apache/nifi/processor/annotation/OnShutdown.java b/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnShutdown.java deleted file mode 100644 index 930a9dffa9f1..000000000000 --- a/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnShutdown.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * 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.processor.annotation; - -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 a method - * should be called whenever the flow is being shutdown. This will be called at - * most once for each processor instance in a process lifetime. - * - * @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-api/src/main/java/org/apache/nifi/processor/annotation/OnStopped.java b/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnStopped.java deleted file mode 100644 index 4fbaf9561889..000000000000 --- a/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnStopped.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * 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.processor.annotation; - -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 a method - * should be called whenever the processor is no longer scheduled to run. - * Methods marked with this annotation will be invoked each time the processor - * 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 processor is no - * longer scheduled to run, see the {@link OnUnscheduled} annotation. - *

- * - * @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-api/src/main/java/org/apache/nifi/processor/annotation/OnUnscheduled.java b/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnUnscheduled.java deleted file mode 100644 index 7b4c1f6585a4..000000000000 --- a/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnUnscheduled.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * 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.processor.annotation; - -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 a method - * should be called whenever the processor is no longer scheduled to run. - * Methods marked with this annotation will be invoked each time the framework - * is notified to stop scheduling the processor. 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. - *

- * - * If any method annotated with this annotation throws, the processor will not - * be scheduled to run. - * - * @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-api/src/main/java/org/apache/nifi/processor/annotation/SideEffectFree.java b/nifi-api/src/main/java/org/apache/nifi/processor/annotation/SideEffectFree.java deleted file mode 100644 index 6c65caa11ac5..000000000000 --- a/nifi-api/src/main/java/org/apache/nifi/processor/annotation/SideEffectFree.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * 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.processor.annotation; - -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 its - * operations on flow files 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 - * 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). - * - * @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-api/src/main/java/org/apache/nifi/processor/annotation/SupportsBatching.java b/nifi-api/src/main/java/org/apache/nifi/processor/annotation/SupportsBatching.java deleted file mode 100644 index e39e21240cb7..000000000000 --- a/nifi-api/src/main/java/org/apache/nifi/processor/annotation/SupportsBatching.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * 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.processor.annotation; - -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 org.apache.nifi.processor.ProcessSession ProcessSession}s' commits, as well as - * allowing the Framework to return the same ProcessSession multiple times from - * subsequent calls to - * {@link org.apache.nifi.processor.ProcessSessionFactory ProcessSessionFactory}.{@link org.apache.nifi.processor.ProcessSessionFactory#createSession() createSession()}. - * - * When this Annotation is used, it is important to note that calls to - * {@link org.apache.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. - * - * @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-api/src/main/java/org/apache/nifi/processor/annotation/Tags.java b/nifi-api/src/main/java/org/apache/nifi/processor/annotation/Tags.java deleted file mode 100644 index 81428d753165..000000000000 --- a/nifi-api/src/main/java/org/apache/nifi/processor/annotation/Tags.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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.processor.annotation; - -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 Processor} in order to associate - * tags with the processor. These tags do not affect the {@link Processor} in - * any way but serve as additional documentation and can be used to sort/filter - * Processors. - * - * @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 { - - public String[] value(); -} diff --git a/nifi-api/src/main/java/org/apache/nifi/processor/annotation/TriggerSerially.java b/nifi-api/src/main/java/org/apache/nifi/processor/annotation/TriggerSerially.java deleted file mode 100644 index 6e80cef48d16..000000000000 --- a/nifi-api/src/main/java/org/apache/nifi/processor/annotation/TriggerSerially.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * 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.processor.annotation; - -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 the - * processor is not thread safe for concurrent execution of its onTrigger() - * method. By default processors are assumed to be thread safe for concurrent - * execution. - * - * @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-api/src/main/java/org/apache/nifi/processor/annotation/TriggerWhenAnyDestinationAvailable.java b/nifi-api/src/main/java/org/apache/nifi/processor/annotation/TriggerWhenAnyDestinationAvailable.java deleted file mode 100644 index 6e01f6c6a0a3..000000000000 --- a/nifi-api/src/main/java/org/apache/nifi/processor/annotation/TriggerWhenAnyDestinationAvailable.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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.processor.annotation; - -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 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. - * - * @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-api/src/main/java/org/apache/nifi/processor/annotation/TriggerWhenEmpty.java b/nifi-api/src/main/java/org/apache/nifi/processor/annotation/TriggerWhenEmpty.java deleted file mode 100644 index d068a9e91d14..000000000000 --- a/nifi-api/src/main/java/org/apache/nifi/processor/annotation/TriggerWhenEmpty.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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.processor.annotation; - -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 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. - * - * @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 { -} diff --git a/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventBuilder.java b/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventBuilder.java index 0ffccd56d47d..d0e20d1ca2fa 100644 --- a/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventBuilder.java +++ b/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventBuilder.java @@ -17,7 +17,6 @@ package org.apache.nifi.provenance; import java.util.Map; -import java.util.Set; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.processor.Processor; @@ -50,15 +49,6 @@ public interface ProvenanceEventBuilder { */ ProvenanceEventBuilder setFlowFileEntryDate(long entryDate); - /** - * Sets the Lineage Identifiers. This is a set of all FlowFile UUID's that - * were involved in making this event occur. - * - * @param lineageIdentifiers of the flowfiles in this event - * @return the builder - */ - ProvenanceEventBuilder setLineageIdentifiers(Set lineageIdentifiers); - /** * Sets the Content Claim that the FlowFile was previously associated with * before this event occurred. diff --git a/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventRecord.java b/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventRecord.java index 53b9c3e0969a..eaa34648c760 100644 --- a/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventRecord.java +++ b/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventRecord.java @@ -18,7 +18,6 @@ import java.util.List; import java.util.Map; -import java.util.Set; /** * Describes an event that happened to a FlowFile. @@ -48,19 +47,6 @@ public interface ProvenanceEventRecord { */ long getLineageStartDate(); - /** - * @return the set of all lineage identifiers that are associated with the - * FlowFile for which this Event was created - * - * @deprecated this collection was erroneously unbounded and caused a lot of OutOfMemoryError problems - * when querying Provenance Events about FlowFiles with many ancestors. This Collection is - * now capped at 100 lineage identifiers. This method was introduced with the idea of providing - * future performance improvements but due to the high cost of heap consumption will not be used - * in such a manner. As a result, this method will be removed in a future release. - */ - @Deprecated - Set getLineageIdentifiers(); - /** * @return the size of the FlowFile to which this Event is associated */ diff --git a/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/LineageNode.java b/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/LineageNode.java index 56e865f1f9b2..5d0956eeae41 100644 --- a/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/LineageNode.java +++ b/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/LineageNode.java @@ -18,13 +18,6 @@ public interface LineageNode { - /** - * @return the identifier of the Clustered NiFi Node that generated the - * event - */ - @Deprecated - String getClusterNodeIdentifier(); - /** * @return the type of the LineageNode */ diff --git a/nifi-api/src/main/java/org/apache/nifi/reporting/ReportingTask.java b/nifi-api/src/main/java/org/apache/nifi/reporting/ReportingTask.java index 3de9b9386b96..780b79eff250 100644 --- a/nifi-api/src/main/java/org/apache/nifi/reporting/ReportingTask.java +++ b/nifi-api/src/main/java/org/apache/nifi/reporting/ReportingTask.java @@ -16,8 +16,8 @@ */ package org.apache.nifi.reporting; +import org.apache.nifi.annotation.lifecycle.OnConfigurationRestored; import org.apache.nifi.components.ConfigurableComponent; -import org.apache.nifi.controller.annotation.OnConfigured; /** * Defines a task that is responsible for reporting status information to @@ -47,7 +47,7 @@ * *

* ReportingTasks may choose to annotate a method with the - * {@link OnConfigured @OnConfigured} annotation. If this is done, that method + * {@link OnConfigurationRestored @OnConfigurationRestored} annotation. If this is done, that method * will be invoked after all properties have been set for the ReportingTask and * before it is scheduled to run. If the method throws an Exception, the * ReportingTask will be Administratively yielded and will not run for the diff --git a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/PlaceholderProvenanceEvent.java b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/PlaceholderProvenanceEvent.java index bc4030237da3..083b13e9d792 100644 --- a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/PlaceholderProvenanceEvent.java +++ b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/PlaceholderProvenanceEvent.java @@ -59,11 +59,6 @@ public long getLineageStartDate() { return 0; } - @Override - public Set getLineageIdentifiers() { - return Collections.emptySet(); - } - @Override public long getFileSize() { return -1L; diff --git a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java index b504b04af5dd..7b0c91f7f844 100644 --- a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java +++ b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java @@ -20,10 +20,8 @@ import java.util.Collections; import java.util.Date; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; @@ -31,7 +29,6 @@ /** * Holder for provenance relevant information - *

*/ public final class StandardProvenanceEventRecord implements ProvenanceEventRecord { @@ -39,7 +36,6 @@ public final class StandardProvenanceEventRecord implements ProvenanceEventRecor private final long entryDate; private final ProvenanceEventType eventType; private final long lineageStartDate; - private final Set lineageIdentifiers; private final String componentId; private final String componentType; private final String transitUri; @@ -91,7 +87,6 @@ private StandardProvenanceEventRecord(final Builder builder) { this.storageFilename = builder.storageFilename; this.eventDuration = builder.eventDuration; this.lineageStartDate = builder.lineageStartDate; - this.lineageIdentifiers = Collections.unmodifiableSet(builder.lineageIdentifiers); previousClaimSection = builder.previousClaimSection; previousClaimContainer = builder.previousClaimContainer; @@ -109,7 +104,7 @@ private StandardProvenanceEventRecord(final Builder builder) { updatedAttributes = builder.updatedAttributes == null ? Collections.emptyMap() : Collections.unmodifiableMap(builder.updatedAttributes); sourceQueueIdentifier = builder.sourceQueueIdentifier; - + } public String getStorageFilename() { @@ -134,11 +129,6 @@ public long getEventTime() { return eventTime; } - @Override - public Set getLineageIdentifiers() { - return lineageIdentifiers; - } - @Override public long getLineageStartDate() { return lineageStartDate; @@ -414,7 +404,6 @@ public static class Builder implements ProvenanceEventBuilder { private long eventTime = System.currentTimeMillis(); private long entryDate; private long lineageStartDate; - private Set lineageIdentifiers = new HashSet<>(); private ProvenanceEventType eventType = null; private String componentId = null; private String componentType = null; @@ -453,7 +442,6 @@ public Builder fromEvent(final ProvenanceEventRecord event) { eventTime = event.getEventTime(); entryDate = event.getFlowFileEntryDate(); lineageStartDate = event.getLineageStartDate(); - lineageIdentifiers = event.getLineageIdentifiers(); eventType = event.getEventType(); componentId = event.getComponentId(); componentType = event.getComponentType(); @@ -498,12 +486,6 @@ public Builder setFlowFileEntryDate(final long entryDate) { return this; } - @Override - public Builder setLineageIdentifiers(final Set lineageIdentifiers) { - this.lineageIdentifiers = lineageIdentifiers; - return this; - } - @Override public Builder setAttributes(final Map previousAttributes, final Map updatedAttributes) { this.previousAttributes = previousAttributes; @@ -541,11 +523,6 @@ public Builder setLineageStartDate(final long startDate) { return this; } - public Builder addLineageIdentifier(final String lineageIdentifier) { - this.lineageIdentifiers.add(lineageIdentifier); - return this; - } - @Override public Builder setEventType(ProvenanceEventType eventType) { this.eventType = eventType; @@ -661,7 +638,6 @@ public Builder setRelationship(final String relationship) { @Override public ProvenanceEventBuilder fromFlowFile(final FlowFile flowFile) { setFlowFileEntryDate(flowFile.getEntryDate()); - setLineageIdentifiers(flowFile.getLineageIdentifiers()); setLineageStartDate(flowFile.getLineageStartDate()); setAttributes(Collections.emptyMap(), flowFile.getAttributes()); uuid = flowFile.getAttribute(CoreAttributes.UUID.key()); diff --git a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/EventNode.java b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/EventNode.java index 1f8d1dcd4650..de516cb9fd70 100644 --- a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/EventNode.java +++ b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/EventNode.java @@ -36,17 +36,6 @@ public String getIdentifier() { return String.valueOf(getEventIdentifier()); } - @Deprecated - @Override - public String getClusterNodeIdentifier() { - return clusterNodeIdentifier; - } - - @Deprecated - public void setClusterNodeIdentifier(final String nodeIdentifier) { - this.clusterNodeIdentifier = nodeIdentifier; - } - @Override public LineageNodeType getNodeType() { return LineageNodeType.PROVENANCE_EVENT_NODE; diff --git a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/FlowFileNode.java b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/FlowFileNode.java index 94e7661aedbe..78813385a7af 100644 --- a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/FlowFileNode.java +++ b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/FlowFileNode.java @@ -22,7 +22,6 @@ public class FlowFileNode implements LineageNode { private final String flowFileUuid; private final long creationTime; - private String clusterNodeIdentifier; public FlowFileNode(final String flowFileUuid, final long flowFileCreationTime) { this.flowFileUuid = requireNonNull(flowFileUuid); @@ -39,12 +38,6 @@ public long getTimestamp() { return creationTime; } - @Deprecated - @Override - public String getClusterNodeIdentifier() { - return clusterNodeIdentifier; - } - @Override public LineageNodeType getNodeType() { return LineageNodeType.FLOWFILE_NODE; diff --git a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java index 05bd0179a2ae..0ede6ccc7921 100644 --- a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java +++ b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java @@ -341,7 +341,6 @@ private String evaluateQueryForEscape(final String queryString, final Map()); Mockito.when(mockFlowFile.getLineageStartDate()).thenReturn(System.currentTimeMillis()); return Query.evaluateExpressions(queryString, mockFlowFile); } diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockFlowFile.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockFlowFile.java index 049c65b052f1..02806b0e8dfc 100644 --- a/nifi-mock/src/main/java/org/apache/nifi/util/MockFlowFile.java +++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockFlowFile.java @@ -45,7 +45,6 @@ public class MockFlowFile implements FlowFileRecord { private final long id; private final long entryDate; - private final Set lineageIdentifiers = new HashSet<>(); private final long creationTime; private boolean penalized = false; @@ -60,7 +59,6 @@ public MockFlowFile(final long id) { final String uuid = UUID.randomUUID().toString(); attributes.put(CoreAttributes.UUID.key(), uuid); - lineageIdentifiers.add(uuid); } public MockFlowFile(final long id, final FlowFile toCopy) { @@ -69,8 +67,6 @@ public MockFlowFile(final long id, final FlowFile toCopy) { final byte[] dataToCopy = ((MockFlowFile) toCopy).data; this.data = new byte[dataToCopy.length]; System.arraycopy(dataToCopy, 0, this.data, 0, dataToCopy.length); - - lineageIdentifiers.addAll(toCopy.getLineageIdentifiers()); } void setPenalized() { @@ -81,11 +77,6 @@ public long getCreationTime() { return creationTime; } - @Override - public Set getLineageIdentifiers() { - return lineageIdentifiers; - } - @Override public long getLineageStartDate() { return entryDate; diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java b/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java index 89793538f819..7d49f0f236e8 100644 --- a/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java +++ b/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java @@ -21,9 +21,7 @@ import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.InputStream; -import java.lang.annotation.Annotation; import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; @@ -87,18 +85,9 @@ public class StandardProcessorTestRunner implements TestRunner { private int numThreads = 1; private final AtomicInteger invocations = new AtomicInteger(0); - private static final Set> deprecatedTypeAnnotations = new HashSet<>(); - private static final Set> deprecatedMethodAnnotations = new HashSet<>(); private final Map controllerServiceLoggers = new HashMap<>(); private final MockComponentLog logger; - static { - // do this in a separate method, just so that we can add a @SuppressWarnings annotation - // because we want to indicate explicitly that we know that we are using deprecated - // classes here. - populateDeprecatedMethods(); - } - StandardProcessorTestRunner(final Processor processor) { this.processor = processor; this.idGenerator = new AtomicLong(0L); @@ -108,8 +97,6 @@ public class StandardProcessorTestRunner implements TestRunner { this.processorStateManager = new MockStateManager(processor); this.context = new MockProcessContext(processor, processorStateManager); - detectDeprecatedAnnotations(processor); - final MockProcessorInitializationContext mockInitContext = new MockProcessorInitializationContext(processor, context); processor.initialize(mockInitContext); logger = mockInitContext.getLogger(); @@ -125,42 +112,6 @@ public class StandardProcessorTestRunner implements TestRunner { ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, processor); } - @SuppressWarnings("deprecation") - private static void populateDeprecatedMethods() { - deprecatedTypeAnnotations.add(org.apache.nifi.processor.annotation.CapabilityDescription.class); - deprecatedTypeAnnotations.add(org.apache.nifi.processor.annotation.EventDriven.class); - deprecatedTypeAnnotations.add(org.apache.nifi.processor.annotation.SideEffectFree.class); - deprecatedTypeAnnotations.add(org.apache.nifi.processor.annotation.SupportsBatching.class); - deprecatedTypeAnnotations.add(org.apache.nifi.processor.annotation.Tags.class); - deprecatedTypeAnnotations.add(org.apache.nifi.processor.annotation.TriggerWhenEmpty.class); - deprecatedTypeAnnotations.add(org.apache.nifi.processor.annotation.TriggerWhenAnyDestinationAvailable.class); - deprecatedTypeAnnotations.add(org.apache.nifi.processor.annotation.TriggerSerially.class); - - deprecatedMethodAnnotations.add(org.apache.nifi.processor.annotation.OnRemoved.class); - deprecatedMethodAnnotations.add(org.apache.nifi.processor.annotation.OnAdded.class); - deprecatedMethodAnnotations.add(org.apache.nifi.processor.annotation.OnScheduled.class); - deprecatedMethodAnnotations.add(org.apache.nifi.processor.annotation.OnShutdown.class); - deprecatedMethodAnnotations.add(org.apache.nifi.processor.annotation.OnStopped.class); - deprecatedMethodAnnotations.add(org.apache.nifi.processor.annotation.OnUnscheduled.class); - } - - private static void detectDeprecatedAnnotations(final Processor processor) { - for (final Class annotationClass : deprecatedTypeAnnotations) { - if (processor.getClass().isAnnotationPresent(annotationClass)) { - Assert.fail("Processor is using deprecated Annotation " + annotationClass.getCanonicalName()); - } - } - - for (final Class annotationClass : deprecatedMethodAnnotations) { - for (final Method method : processor.getClass().getMethods()) { - if (method.isAnnotationPresent(annotationClass)) { - Assert.fail("Processor is using deprecated Annotation " + annotationClass.getCanonicalName() + " for method " + method); - } - } - } - - } - @Override public void setValidateExpressionUsage(final boolean validate) { context.setValidateExpressionUsage(validate); diff --git a/nifi-mock/src/test/java/org/apache/nifi/util/TestStandardProcessorTestRunner.java b/nifi-mock/src/test/java/org/apache/nifi/util/TestStandardProcessorTestRunner.java index 342b01699c26..0e41e89f26d3 100644 --- a/nifi-mock/src/test/java/org/apache/nifi/util/TestStandardProcessorTestRunner.java +++ b/nifi-mock/src/test/java/org/apache/nifi/util/TestStandardProcessorTestRunner.java @@ -29,7 +29,6 @@ import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; -import org.junit.Ignore; import org.junit.Test; public class TestStandardProcessorTestRunner { @@ -111,39 +110,6 @@ public void testAllFlowFilesContainAttribute() { runner.assertAllFlowFilesContainAttribute(AddAttributeProcessor.KEY); } - @Test(expected = AssertionError.class) - @Ignore("This should not be enabled until we actually fail processor unit tests for using deprecated methods") - public void testFailOnDeprecatedTypeAnnotation() { - new StandardProcessorTestRunner(new DeprecatedAnnotation()); - } - - @Test - @Ignore("This should not be enabled until we actually fail processor unit tests for using deprecated methods") - public void testDoesNotFailOnNonDeprecatedTypeAnnotation() { - new StandardProcessorTestRunner(new NewAnnotation()); - } - - @Test(expected = AssertionError.class) - @Ignore("This should not be enabled until we actually fail processor unit tests for using deprecated methods") - public void testFailOnDeprecatedMethodAnnotation() { - new StandardProcessorTestRunner(new DeprecatedMethodAnnotation()); - } - - @Test - @Ignore("This should not be enabled until we actually fail processor unit tests for using deprecated methods") - public void testDoesNotFailOnNonDeprecatedMethodAnnotation() { - new StandardProcessorTestRunner(new NewMethodAnnotation()); - } - - @SuppressWarnings("deprecation") - @org.apache.nifi.processor.annotation.Tags({"deprecated"}) - private static class DeprecatedAnnotation extends AbstractProcessor { - - @Override - public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { - } - } - @org.apache.nifi.annotation.documentation.Tags({"deprecated"}) private static class NewAnnotation extends AbstractProcessor { @@ -164,19 +130,6 @@ public void onTrigger(ProcessContext context, ProcessSession session) throws Pro } } - private static class DeprecatedMethodAnnotation extends AbstractProcessor { - - @SuppressWarnings("deprecation") - @org.apache.nifi.processor.annotation.OnScheduled - public void dummy() { - - } - - @Override - public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { - } - } - private static class ProcessorWithOnStop extends AbstractProcessor { private int callsWithContext = 0; diff --git a/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/util/FlowFileEvent.java b/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/util/FlowFileEvent.java index c552d4de299b..d68294625149 100644 --- a/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/util/FlowFileEvent.java +++ b/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/util/FlowFileEvent.java @@ -29,7 +29,6 @@ import static org.apache.nifi.processors.flume.util.FlowFileEventConstants.ENTRY_DATE_HEADER; import static org.apache.nifi.processors.flume.util.FlowFileEventConstants.ID_HEADER; import static org.apache.nifi.processors.flume.util.FlowFileEventConstants.LAST_QUEUE_DATE_HEADER; -import static org.apache.nifi.processors.flume.util.FlowFileEventConstants.LINEAGE_IDENTIFIERS_HEADER; import static org.apache.nifi.processors.flume.util.FlowFileEventConstants.LINEAGE_START_DATE_HEADER; import static org.apache.nifi.processors.flume.util.FlowFileEventConstants.SIZE_HEADER; @@ -65,11 +64,6 @@ public Map getHeaders() { headers.put(ENTRY_DATE_HEADER, Long.toString(flowFile.getEntryDate())); headers.put(ID_HEADER, Long.toString(flowFile.getId())); headers.put(LAST_QUEUE_DATE_HEADER, Long.toString(flowFile.getLastQueueDate())); - int i = 0; - for (String lineageIdentifier : flowFile.getLineageIdentifiers()) { - headers.put(LINEAGE_IDENTIFIERS_HEADER + "." + i, lineageIdentifier); - i++; - } headers.put(LINEAGE_START_DATE_HEADER, Long.toString(flowFile.getLineageStartDate())); headers.put(SIZE_HEADER, Long.toString(flowFile.getSize())); headersLoaded = true; diff --git a/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/util/FlowFileEventConstants.java b/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/util/FlowFileEventConstants.java index c9650c1e8735..2c0dd9cbabea 100644 --- a/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/util/FlowFileEventConstants.java +++ b/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/util/FlowFileEventConstants.java @@ -28,9 +28,6 @@ public class FlowFileEventConstants { // FlowFile#getLastQueueDate(); public static final String LAST_QUEUE_DATE_HEADER = "nifi.last.queue.date"; - // FlowFile#getLineageIdentifiers(); - public static final String LINEAGE_IDENTIFIERS_HEADER = "nifi.lineage.identifiers"; - // FlowFile#getLineageStartDate(); public static final String LINEAGE_START_DATE_HEADER = "nifi.lineage.start.date"; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ControllerServiceInitializer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ControllerServiceInitializer.java index 4d1651e44093..a6c397a8a57f 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ControllerServiceInitializer.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ControllerServiceInitializer.java @@ -51,7 +51,7 @@ public void teardown(ConfigurableComponent component) { final ComponentLog logger = new MockComponentLogger(); final MockConfigurationContext context = new MockConfigurationContext(); - ReflectionUtils.quietlyInvokeMethodsWithAnnotations(OnShutdown.class, org.apache.nifi.processor.annotation.OnShutdown.class, controllerService, logger, context); + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, controllerService, logger, context); } } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ProcessorInitializer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ProcessorInitializer.java index 7a66f7290136..745c27c53c1f 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ProcessorInitializer.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ProcessorInitializer.java @@ -49,7 +49,7 @@ public void teardown(ConfigurableComponent component) { final ComponentLog logger = new MockComponentLogger(); final MockProcessContext context = new MockProcessContext(); - ReflectionUtils.quietlyInvokeMethodsWithAnnotations(OnShutdown.class, org.apache.nifi.processor.annotation.OnShutdown.class, processor, logger, context); + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, processor, logger, context); } } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ReportingTaskingInitializer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ReportingTaskingInitializer.java index 32e878c92cc4..3cf86c9b190b 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ReportingTaskingInitializer.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ReportingTaskingInitializer.java @@ -48,7 +48,7 @@ public void teardown(ConfigurableComponent component) { try (NarCloseable narCloseable = NarCloseable.withNarLoader()) { final MockConfigurationContext context = new MockConfigurationContext(); - ReflectionUtils.quietlyInvokeMethodsWithAnnotations(OnShutdown.class, org.apache.nifi.processor.annotation.OnShutdown.class, reportingTask, new MockComponentLogger(), context); + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, reportingTask, new MockComponentLogger(), context); } } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/util/ReflectionUtils.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/util/ReflectionUtils.java index 94fc1f3a03f6..1d3de3645ccb 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/util/ReflectionUtils.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/util/ReflectionUtils.java @@ -19,121 +19,115 @@ 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.apache.nifi.logging.ComponentLog; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * This class is a copy of org.apache.nifi.util.ReflectionUtils. Ultimately the documentation generation - * component should be moved to a place where it can depend on this directly instead of copying it in. + * This class is a copy of org.apache.nifi.util.ReflectionUtils. Ultimately the + * documentation generation component should be moved to a place where it can + * depend on this directly instead of copying it in. * * */ public class ReflectionUtils { private final static Logger LOG = LoggerFactory.getLogger(ReflectionUtils.class); + /** - * 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. + * 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 + * 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 preferred - * @param alternateAnnotation alternate + * @param annotation annotation * @param instance instance - * @param logger the ComponentLog to use for logging any errors. If null, will use own logger, but that will not generate bulletins or easily tie to the Processor's log messages. + * @param logger the ComponentLog to use for logging any errors. If null, + * will use own logger, but that will not generate bulletins or easily tie + * to the Processor's log messages. * @param args 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. + * @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 quietlyInvokeMethodsWithAnnotations( - final Class preferredAnnotation, final Class alternateAnnotation, final Object instance, final ComponentLog logger, final Object... args) { - final List> annotationClasses = new ArrayList<>(alternateAnnotation == null ? 1 : 2); - annotationClasses.add(preferredAnnotation); - if (alternateAnnotation != null) { - annotationClasses.add(alternateAnnotation); - } + public static boolean quietlyInvokeMethodsWithAnnotation( + final Class annotation, final Object instance, final ComponentLog logger, final Object... args) { - boolean annotationFound = false; - for (final Class annotationClass : annotationClasses) { - if (annotationFound) { - break; - } + for (final Method method : instance.getClass().getMethods()) { + if (method.isAnnotationPresent(annotation)) { - for (final Method method : instance.getClass().getMethods()) { - if (method.isAnnotationPresent(annotationClass)) { - annotationFound = true; + final boolean isAccessible = method.isAccessible(); + method.setAccessible(true); - final boolean isAccessible = method.isAccessible(); - method.setAccessible(true); + try { + final Class[] argumentTypes = method.getParameterTypes(); + if (argumentTypes.length > args.length) { + if (logger == null) { + LOG.error("Unable to invoke method {} on {} because method expects {} parameters but only {} were given", + new Object[]{method.getName(), instance, argumentTypes.length, args.length}); + } else { + logger.error("Unable to invoke method {} on {} because method expects {} parameters but only {} were given", + new Object[]{method.getName(), instance, argumentTypes.length, args.length}); + } - try { - final Class[] argumentTypes = method.getParameterTypes(); - if (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())) { if (logger == null) { - LOG.error("Unable to invoke method {} on {} because method expects {} parameters but only {} were given", - new Object[]{method.getName(), instance, argumentTypes.length, args.length}); + 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()}); } else { - logger.error("Unable to invoke method {} on {} because method expects {} parameters but only {} were given", - new Object[]{method.getName(), instance, argumentTypes.length, args.length}); + logger.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; } + } - for (int i = 0; i < argumentTypes.length; i++) { - final Class argType = argumentTypes[i]; - if (!argType.isAssignableFrom(args[i].getClass())) { - if (logger == null) { - 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()}); - } else { - logger.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; - } - } - - 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 InvocationTargetException ite) { - if (logger == null) { - LOG.error("Unable to invoke method {} on {} due to {}", new Object[]{method.getName(), instance, ite.getCause()}); - LOG.error("", ite.getCause()); - } else { - logger.error("Unable to invoke method {} on {} due to {}", new Object[]{method.getName(), instance, ite.getCause()}); - } - } catch (final IllegalAccessException | IllegalArgumentException t) { - if (logger == null) { - LOG.error("Unable to invoke method {} on {} due to {}", new Object[]{method.getName(), instance, t}); - LOG.error("", t); - } else { - logger.error("Unable to invoke method {} on {} due to {}", new Object[]{method.getName(), instance, t}); + 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]; } - return false; + method.invoke(instance, argsToPass); + } + } catch (final InvocationTargetException ite) { + if (logger == null) { + LOG.error("Unable to invoke method {} on {} due to {}", new Object[]{method.getName(), instance, ite.getCause()}); + LOG.error("", ite.getCause()); + } else { + logger.error("Unable to invoke method {} on {} due to {}", new Object[]{method.getName(), instance, ite.getCause()}); } - } finally { - if (!isAccessible) { - method.setAccessible(false); + } catch (final IllegalAccessException | IllegalArgumentException t) { + if (logger == null) { + LOG.error("Unable to invoke method {} on {} due to {}", new Object[]{method.getName(), instance, t}); + LOG.error("", t); + } else { + logger.error("Unable to invoke method {} on {} due to {}", new Object[]{method.getName(), instance, t}); } + + return false; + } + } finally { + if (!isAccessible) { + method.setAccessible(false); } } } } + return true; } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FileSystemSwapManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FileSystemSwapManager.java index 5234f0ea95de..5a33eecbfdfd 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FileSystemSwapManager.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FileSystemSwapManager.java @@ -33,10 +33,8 @@ import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.UUID; import java.util.regex.Pattern; @@ -74,7 +72,7 @@ public class FileSystemSwapManager implements FlowFileSwapManager { private static final Pattern SWAP_FILE_PATTERN = Pattern.compile("\\d+-.+\\.swap"); private static final Pattern TEMP_SWAP_FILE_PATTERN = Pattern.compile("\\d+-.+\\.swap\\.part"); - public static final int SWAP_ENCODING_VERSION = 9; + public static final int SWAP_ENCODING_VERSION = 10; public static final String EVENT_CATEGORY = "Swap FlowFiles"; private static final Logger logger = LoggerFactory.getLogger(FileSystemSwapManager.class); @@ -320,13 +318,6 @@ public static int serializeFlowFiles(final List toSwap, final Fl for (final FlowFileRecord flowFile : toSwap) { out.writeLong(flowFile.getId()); out.writeLong(flowFile.getEntryDate()); - - final Set lineageIdentifiers = flowFile.getLineageIdentifiers(); - out.writeInt(lineageIdentifiers.size()); - for (final String lineageId : lineageIdentifiers) { - out.writeUTF(lineageId); - } - out.writeLong(flowFile.getLineageStartDate()); out.writeLong(flowFile.getLineageStartIndex()); out.writeLong(flowFile.getLastQueueDate()); @@ -443,12 +434,12 @@ private static SwapContents deserializeFlowFiles(final DataInputStream in, final if (serializationVersion > 1) { // Lineage information was added in version 2 - final int numLineageIdentifiers = in.readInt(); - final Set lineageIdentifiers = new HashSet<>(numLineageIdentifiers); - for (int lineageIdIdx = 0; lineageIdIdx < numLineageIdentifiers; lineageIdIdx++) { - lineageIdentifiers.add(in.readUTF()); + if(serializationVersion < 10){ + final int numLineageIdentifiers = in.readInt(); + for (int lineageIdIdx = 0; lineageIdIdx < numLineageIdentifiers; lineageIdIdx++) { + in.readUTF(); //skip each identifier + } } - ffBuilder.lineageIdentifiers(lineageIdentifiers); // version 9 adds in a 'lineage start index' final long lineageStartDate = in.readLong(); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java index 0c90c5058fd5..76b946ffb7da 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java @@ -1019,7 +1019,7 @@ public ProcessorNode createProcessor(final String type, String id, final boolean if (firstTimeAdded) { try (final NarCloseable x = NarCloseable.withNarLoader()) { - ReflectionUtils.invokeMethodsWithAnnotations(OnAdded.class, org.apache.nifi.processor.annotation.OnAdded.class, processor); + ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, processor); } catch (final Exception e) { logRepository.removeObserver(StandardProcessorNode.BULLETIN_OBSERVER_ID); throw new ComponentLifeCycleException("Failed to invoke @OnAdded methods of " + procNode.getProcessor(), e); @@ -3662,11 +3662,6 @@ public ProvenanceEventRecord replayFlowFile(final ProvenanceEventRecord event, f final String parentUUID = event.getFlowFileUuid(); - // Create the FlowFile Record - final Set lineageIdentifiers = new HashSet<>(); - lineageIdentifiers.addAll(event.getLineageIdentifiers()); - lineageIdentifiers.add(parentUUID); - final String newFlowFileUUID = UUID.randomUUID().toString(); // We need to create a new FlowFile by populating it with information from the @@ -3685,7 +3680,6 @@ public ProvenanceEventRecord replayFlowFile(final ProvenanceEventRecord event, f .contentClaimOffset(0L) // use 0 because we used the content claim offset in the Content Claim itself .entryDate(System.currentTimeMillis()) .id(flowFileRepository.getNextFlowFileSequence()) - .lineageIdentifiers(lineageIdentifiers) .lineageStart(event.getLineageStartDate(), 0L) .size(contentSize.longValue()) // Create a new UUID and add attributes indicating that this is a replay diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java index 3ea2b6bce8f1..b232f265388a 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java @@ -151,7 +151,7 @@ public StandardProcessorNode(final Processor processor, final String uuid, identifier = new AtomicReference<>(uuid); destinations = new HashMap<>(); connections = new HashMap<>(); - incomingConnectionsRef = new AtomicReference>(new ArrayList()); + incomingConnectionsRef = new AtomicReference<>(new ArrayList<>()); lossTolerant = new AtomicBoolean(false); final Set emptySetOfRelationships = new HashSet<>(); undefinedRelationshipsToTerminate = new AtomicReference<>(emptySetOfRelationships); @@ -169,20 +169,12 @@ public StandardProcessorNode(final Processor processor, final String uuid, penalizationPeriod = new AtomicReference<>(DEFAULT_PENALIZATION_PERIOD); 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); - eventDrivenSupported = (procClass.isAnnotationPresent(EventDriven.class) - || procClass.isAnnotationPresent(org.apache.nifi.processor.annotation.EventDriven.class)) - && !triggeredSerially && !triggerWhenEmpty; + triggerWhenEmpty = procClass.isAnnotationPresent(TriggerWhenEmpty.class); + sideEffectFree = procClass.isAnnotationPresent(SideEffectFree.class); + batchSupported = procClass.isAnnotationPresent(SupportsBatching.class); + triggeredSerially = procClass.isAnnotationPresent(TriggerSerially.class); + triggerWhenAnyDestinationAvailable = procClass.isAnnotationPresent(TriggerWhenAnyDestinationAvailable.class); + eventDrivenSupported = procClass.isAnnotationPresent(EventDriven.class) && !triggeredSerially && !triggerWhenEmpty; final boolean inputRequirementPresent = procClass.isAnnotationPresent(InputRequirement.class); if (inputRequirementPresent) { @@ -371,20 +363,12 @@ public Set getAutoTerminatedRelationships() { * @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); 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; } @@ -1254,8 +1238,7 @@ public void run() { @Override public Void call() throws Exception { try (final NarCloseable nc = NarCloseable.withNarLoader()) { - ReflectionUtils.invokeMethodsWithAnnotations(OnScheduled.class, - org.apache.nifi.processor.annotation.OnScheduled.class, processor, processContext); + ReflectionUtils.invokeMethodsWithAnnotation(OnScheduled.class, processor, processContext); return null; } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java index 9117a1626023..bcb3feb30e91 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java @@ -30,16 +30,12 @@ import org.apache.nifi.controller.ReportingTaskNode; import org.apache.nifi.controller.ScheduledState; import org.apache.nifi.controller.ValidationContextFactory; -import org.apache.nifi.controller.annotation.OnConfigured; -import org.apache.nifi.controller.exception.ComponentLifeCycleException; import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.controller.service.ControllerServiceProvider; import org.apache.nifi.controller.service.StandardConfigurationContext; -import org.apache.nifi.nar.NarCloseable; import org.apache.nifi.reporting.ReportingTask; import org.apache.nifi.scheduling.SchedulingStrategy; import org.apache.nifi.util.FormatUtils; -import org.apache.nifi.util.ReflectionUtils; public abstract class AbstractReportingTaskNode extends AbstractConfiguredComponent implements ReportingTaskNode { @@ -138,30 +134,11 @@ public void setScheduledState(final ScheduledState state) { @Override public void setProperty(final String name, final String value) { super.setProperty(name, value); - - onConfigured(); } @Override public boolean removeProperty(String name) { - final boolean removed = super.removeProperty(name); - if (removed) { - onConfigured(); - } - - return removed; - } - - @SuppressWarnings("deprecation") - private void onConfigured() { - // We need to invoke any method annotation with the OnConfigured annotation in order to - // maintain backward compatibility. This will be removed when we remove the old, deprecated annotations. - try (final NarCloseable x = NarCloseable.withNarLoader()) { - final ConfigurationContext configContext = new StandardConfigurationContext(this, serviceLookup, getSchedulingPeriod()); - ReflectionUtils.invokeMethodsWithAnnotation(OnConfigured.class, reportingTask, configContext); - } catch (final Exception e) { - throw new ComponentLifeCycleException("Failed to invoke On-Configured Lifecycle methods of " + reportingTask, e); - } + return super.removeProperty(name); } public boolean isDisabled() { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardFlowFileRecord.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardFlowFileRecord.java index cd1ba141099f..607ccfd470a6 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardFlowFileRecord.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardFlowFileRecord.java @@ -16,7 +16,6 @@ */ package org.apache.nifi.controller.repository; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -43,13 +42,11 @@ * */ public final class StandardFlowFileRecord implements FlowFile, FlowFileRecord { - private static final int MAX_LINEAGE_IDENTIFIERS = 100; private final long id; private final long entryDate; private final long lineageStartDate; private final long lineageStartIndex; - private final Set lineageIdentifiers; private final long size; private final long penaltyExpirationMs; private final Map attributes; @@ -64,7 +61,6 @@ private StandardFlowFileRecord(final Builder builder) { this.entryDate = builder.bEntryDate; this.lineageStartDate = builder.bLineageStartDate; this.lineageStartIndex = builder.bLineageStartIndex; - this.lineageIdentifiers = builder.bLineageIdentifiers; this.penaltyExpirationMs = builder.bPenaltyExpirationMs; this.size = builder.bSize; this.claim = builder.bClaim; @@ -83,11 +79,6 @@ public long getEntryDate() { return entryDate; } - @Override - public Set getLineageIdentifiers() { - return Collections.unmodifiableSet(lineageIdentifiers); - } - @Override public long getLineageStartDate() { return lineageStartDate; @@ -196,25 +187,6 @@ public Builder id(final long id) { return this; } - public Builder lineageIdentifiers(final Collection lineageIdentifiers) { - if (null != lineageIdentifiers) { - bLineageIdentifiers.clear(); - - if (lineageIdentifiers.size() > MAX_LINEAGE_IDENTIFIERS) { - int i = 0; - for (final String id : lineageIdentifiers) { - bLineageIdentifiers.add(id); - if (i++ >= MAX_LINEAGE_IDENTIFIERS) { - break; - } - } - } else { - bLineageIdentifiers.addAll(lineageIdentifiers); - } - } - return this; - } - public Builder entryDate(final long epochMs) { bEntryDate = epochMs; return this; @@ -330,7 +302,6 @@ public Builder fromFlowFile(final FlowFileRecord specFlowFile) { bLineageStartDate = specFlowFile.getLineageStartDate(); bLineageStartIndex = specFlowFile.getLineageStartIndex(); bLineageIdentifiers.clear(); - bLineageIdentifiers.addAll(specFlowFile.getLineageIdentifiers()); bPenaltyExpirationMs = specFlowFile.getPenaltyExpirationMillis(); bSize = specFlowFile.getSize(); bAttributes.putAll(specFlowFile.getAttributes()); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java index b5da0720ed93..1e2d63491934 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java @@ -1414,7 +1414,6 @@ private void registerForkEvent(final FlowFile parent, final FlowFile child) { eventBuilder.setEventType(ProvenanceEventType.FORK); eventBuilder.setFlowFileEntryDate(parent.getEntryDate()); - eventBuilder.setLineageIdentifiers(parent.getLineageIdentifiers()); eventBuilder.setLineageStartDate(parent.getLineageStartDate()); eventBuilder.setFlowFileUUID(parent.getAttribute(CoreAttributes.UUID.key())); @@ -2618,9 +2617,6 @@ public FlowFile create(final FlowFile parent) { newAttributes.put(key, value); } - final Set lineageIdentifiers = new HashSet<>(parent.getLineageIdentifiers()); - lineageIdentifiers.add(parent.getAttribute(CoreAttributes.UUID.key())); - fFileBuilder.lineageIdentifiers(lineageIdentifiers); fFileBuilder.lineageStart(parent.getLineageStartDate(), parent.getLineageStartIndex()); fFileBuilder.addAttributes(newAttributes); @@ -2646,8 +2642,6 @@ public FlowFile create(final Collection parents) { long lineageStartDate = 0L; final Set lineageIdentifiers = new HashSet<>(); for (final FlowFile parent : parents) { - lineageIdentifiers.addAll(parent.getLineageIdentifiers()); - lineageIdentifiers.add(parent.getAttribute(CoreAttributes.UUID.key())); final long parentLineageStartDate = parent.getLineageStartDate(); if (lineageStartDate == 0L || parentLineageStartDate < lineageStartDate) { @@ -2669,7 +2663,6 @@ public FlowFile create(final Collection parents) { final FlowFileRecord fFile = new StandardFlowFileRecord.Builder().id(context.getNextFlowFileSequence()) .addAttributes(newAttributes) - .lineageIdentifiers(lineageIdentifiers) .lineageStart(lineageStartDate, lineageStartIndex) .build(); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java index 0f40cbb61209..dae1cff77669 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java @@ -388,7 +388,7 @@ public int checkpoint() throws IOException { } private static class WriteAheadRecordSerde implements SerDe { - private static final int CURRENT_ENCODING_VERSION = 8; + private static final int CURRENT_ENCODING_VERSION = 9; public static final byte ACTION_CREATE = 0; public static final byte ACTION_UPDATE = 1; @@ -467,13 +467,6 @@ public void serializeEdit(final RepositoryRecord previousRecordState, final Repo out.writeLong(getRecordIdentifier(record)); out.writeLong(flowFile.getEntryDate()); - - final Set lineageIdentifiers = flowFile.getLineageIdentifiers(); - out.writeInt(lineageIdentifiers.size()); - for (final String lineageId : lineageIdentifiers) { - out.writeUTF(lineageId); - } - out.writeLong(flowFile.getLineageStartDate()); out.writeLong(flowFile.getLineageStartIndex()); @@ -549,13 +542,12 @@ public RepositoryRecord deserializeEdit(final DataInputStream in, final Map 1) { // read the lineage identifiers and lineage start date, which were added in version 2. - final int numLineageIds = in.readInt(); - final Set lineageIdentifiers = new HashSet<>(numLineageIds); - for (int i = 0; i < numLineageIds; i++) { - lineageIdentifiers.add(in.readUTF()); + if(version < 9){ + final int numLineageIds = in.readInt(); + for (int i = 0; i < numLineageIds; i++) { + in.readUTF(); //skip identifiers + } } - ffBuilder.lineageIdentifiers(lineageIdentifiers); - final long lineageStartDate = in.readLong(); final long lineageStartIndex; if (version > 7) { @@ -661,12 +653,12 @@ public StandardRepositoryRecord deserializeRecord(final DataInputStream in, fina if (version > 1) { // read the lineage identifiers and lineage start date, which were added in version 2. - final int numLineageIds = in.readInt(); - final Set lineageIdentifiers = new HashSet<>(numLineageIds); - for (int i = 0; i < numLineageIds; i++) { - lineageIdentifiers.add(in.readUTF()); + if(version < 9) { + final int numLineageIds = in.readInt(); + for (int i = 0; i < numLineageIds; i++) { + in.readUTF(); //skip identifiers + } } - ffBuilder.lineageIdentifiers(lineageIdentifiers); final long lineageStartDate = in.readLong(); final long lineageStartIndex; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java index c47d6aa878cf..d3410cca50f5 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java @@ -303,7 +303,7 @@ private void trigger(final Connectable worker, final ScheduleState scheduleState } finally { if (!scheduleState.isScheduled() && scheduleState.getActiveThreadCount() == 1 && scheduleState.mustCallOnStoppedMethods()) { try (final NarCloseable x = NarCloseable.withNarLoader()) { - ReflectionUtils.quietlyInvokeMethodsWithAnnotations(OnStopped.class, org.apache.nifi.processor.annotation.OnStopped.class, worker, processContext); + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, worker, processContext); } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java index 7856dcd054ed..0c1979ced195 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java @@ -45,11 +45,8 @@ import org.apache.nifi.controller.ConfiguredComponent; import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.ValidationContextFactory; -import org.apache.nifi.controller.annotation.OnConfigured; -import org.apache.nifi.controller.exception.ComponentLifeCycleException; import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.logging.ComponentLog; -import org.apache.nifi.nar.NarCloseable; import org.apache.nifi.processor.SimpleProcessLogger; import org.apache.nifi.util.ReflectionUtils; import org.slf4j.Logger; @@ -203,27 +200,11 @@ public void verifyModifiable() throws IllegalStateException { @Override public void setProperty(final String name, final String value) { super.setProperty(name, value); - onConfigured(); } @Override public boolean removeProperty(String name) { - final boolean removed = super.removeProperty(name); - if (removed) { - onConfigured(); - } - - return removed; - } - - @SuppressWarnings("deprecation") - private void onConfigured() { - try (final NarCloseable x = NarCloseable.withNarLoader()) { - final ConfigurationContext configContext = new StandardConfigurationContext(this, serviceProvider, null); - ReflectionUtils.invokeMethodsWithAnnotation(OnConfigured.class, implementation, configContext); - } catch (final Exception e) { - throw new ComponentLifeCycleException("Failed to invoke On-Configured Lifecycle methods of " + implementation, e); - } + return super.removeProperty(name); } @Override diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java index 04e3f6057f2c..8bc7d994a3aa 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java @@ -95,7 +95,7 @@ public Boolean call() { } finally { if (!scheduleState.isScheduled() && scheduleState.getActiveThreadCount() == 1 && scheduleState.mustCallOnStoppedMethods()) { try (final NarCloseable x = NarCloseable.withNarLoader()) { - ReflectionUtils.quietlyInvokeMethodsWithAnnotations(OnStopped.class, org.apache.nifi.processor.annotation.OnStopped.class, connectable, processContext); + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, connectable, processContext); } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java index 77f60b540ee1..1dc08c2b2f6b 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java @@ -52,9 +52,7 @@ public synchronized void run() { // invoke the OnStopped methods if (!scheduleState.isScheduled() && scheduleState.getActiveThreadCount() == 1 && scheduleState.mustCallOnStoppedMethods()) { try (final NarCloseable x = NarCloseable.withNarLoader()) { - ReflectionUtils.quietlyInvokeMethodsWithAnnotations( - OnStopped.class, org.apache.nifi.processor.annotation.OnStopped.class, - taskNode.getReportingTask(), taskNode.getConfigurationContext()); + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, taskNode.getReportingTask(), taskNode.getConfigurationContext()); } } } finally { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java index a931841294fd..c8d9d2523458 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java @@ -338,7 +338,7 @@ private void shutdown(final ProcessGroup procGroup) { for (final ProcessorNode node : procGroup.getProcessors()) { try (final NarCloseable x = NarCloseable.withNarLoader()) { final StandardProcessContext processContext = new StandardProcessContext(node, controllerServiceProvider, encryptor, getStateManager(node.getIdentifier())); - ReflectionUtils.quietlyInvokeMethodsWithAnnotations(OnShutdown.class, org.apache.nifi.processor.annotation.OnShutdown.class, node.getProcessor(), processContext); + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, node.getProcessor(), processContext); } } @@ -704,7 +704,7 @@ public void removeProcessor(final ProcessorNode processor) { try (final NarCloseable x = NarCloseable.withNarLoader()) { final StandardProcessContext processContext = new StandardProcessContext(processor, controllerServiceProvider, encryptor, getStateManager(processor.getIdentifier())); - ReflectionUtils.quietlyInvokeMethodsWithAnnotations(OnRemoved.class, org.apache.nifi.processor.annotation.OnRemoved.class, processor.getProcessor(), processContext); + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, processor.getProcessor(), processContext); } catch (final Exception e) { throw new ComponentLifeCycleException("Failed to invoke 'OnRemoved' methods of " + processor, e); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFileSystemSwapManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFileSystemSwapManager.java index c27fad3b23ae..50bc874797a2 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFileSystemSwapManager.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFileSystemSwapManager.java @@ -32,7 +32,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; @@ -110,7 +109,6 @@ public void testRoundTripSerializeDeserialize() throws IOException { assertEquals(pre.getContentClaimOffset(), post.getContentClaimOffset()); assertEquals(pre.getEntryDate(), post.getEntryDate()); assertEquals(pre.getLastQueueDate(), post.getLastQueueDate()); - assertEquals(pre.getLineageIdentifiers(), post.getLineageIdentifiers()); assertEquals(pre.getLineageStartDate(), post.getLineageStartDate()); assertEquals(pre.getPenaltyExpirationMillis(), post.getPenaltyExpirationMillis()); } @@ -198,11 +196,6 @@ public Long getLastQueueDate() { return lastQueueDate; } - @Override - public Set getLineageIdentifiers() { - return Collections.emptySet(); - } - @Override public boolean isPenalized() { return false; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardFlowFileQueue.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardFlowFileQueue.java index 85d983800ef3..6e368de16bed 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardFlowFileQueue.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardFlowFileQueue.java @@ -524,7 +524,7 @@ public void enableIncompleteSwapFileException(final int flowFilesToInclude) { public String swapOut(List flowFiles, FlowFileQueue flowFileQueue) throws IOException { swapOutCalledCount++; final String location = UUID.randomUUID().toString(); - swappedOut.put(location, new ArrayList(flowFiles)); + swappedOut.put(location, new ArrayList<>(flowFiles)); return location; } @@ -560,7 +560,7 @@ public SwapContents swapIn(String swapLocation, FlowFileQueue flowFileQueue) thr @Override public List recoverSwapLocations(FlowFileQueue flowFileQueue) throws IOException { - return new ArrayList(swappedOut.keySet()); + return new ArrayList<>(swappedOut.keySet()); } @Override @@ -610,7 +610,7 @@ public TestFlowFile() { } public TestFlowFile(final long size) { - this(new HashMap(), size); + this(new HashMap<>(), size); } public TestFlowFile(final Map attributes, final long size) { @@ -647,11 +647,6 @@ public Long getLastQueueDate() { return null; } - @Override - public Set getLineageIdentifiers() { - return Collections.emptySet(); - } - @Override public boolean isPenalized() { return false; diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java index 3b827eba4caf..c6c4b159bf83 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java @@ -118,7 +118,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository private static final String FILE_EXTENSION = ".prov"; private static final String TEMP_FILE_SUFFIX = ".prov.part"; private static final long PURGE_EVENT_MILLISECONDS = 2500L; //Determines the frequency over which the task to delete old events will occur - public static final int SERIALIZATION_VERSION = 8; + public static final int SERIALIZATION_VERSION = 9; public static final Pattern NUMBER_PATTERN = Pattern.compile("\\d+"); public static final Pattern INDEX_PATTERN = Pattern.compile("index-\\d+"); public static final Pattern LOG_FILENAME_PATTERN = Pattern.compile("(\\d+).*\\.prov"); diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/StandardRecordReader.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/StandardRecordReader.java index a9d7bf7c91ec..2db9ed32f0e5 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/StandardRecordReader.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/StandardRecordReader.java @@ -23,9 +23,7 @@ import java.nio.charset.StandardCharsets; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.Map; -import java.util.Set; import java.util.UUID; import java.util.regex.Pattern; import java.util.zip.GZIPInputStream; @@ -92,8 +90,8 @@ public StandardRecordReader(final InputStream in, final String filename, final T final int serializationVersion = dis.readInt(); headerLength = repoClassName.getBytes(StandardCharsets.UTF_8).length + 2 + 4; // 2 bytes for string length, 4 for integer. - if (serializationVersion < 1 || serializationVersion > 8) { - throw new IllegalArgumentException("Unable to deserialize record because the version is " + serializationVersion + " and supported versions are 1-8"); + if (serializationVersion < 1 || serializationVersion > 9) { + throw new IllegalArgumentException("Unable to deserialize record because the version is " + serializationVersion + " and supported versions are 1-9"); } this.serializationVersion = serializationVersion; @@ -252,7 +250,6 @@ private StandardProvenanceEventRecord readPreVersion6Record() throws IOException final Map attrs = readAttributes(dis, false); builder.setFlowFileEntryDate(System.currentTimeMillis()); - builder.setLineageIdentifiers(Collections.emptySet()); builder.setLineageStartDate(-1L); builder.setAttributes(Collections.emptyMap(), attrs); builder.setCurrentContentClaim(null, null, null, null, fileSize); @@ -288,10 +285,11 @@ public StandardProvenanceEventRecord nextRecord() throws IOException { final Long flowFileEntryDate = dis.readLong(); builder.setEventDuration(dis.readLong()); - final Set lineageIdentifiers = new HashSet<>(); - final int numLineageIdentifiers = dis.readInt(); - for (int i = 0; i < numLineageIdentifiers; i++) { - lineageIdentifiers.add(readUUID(dis)); + if (serializationVersion < 9){ + final int numLineageIdentifiers = dis.readInt(); + for (int i = 0; i < numLineageIdentifiers; i++) { + readUUID(dis); //skip identifiers + } } final long lineageStartDate = dis.readLong(); @@ -358,7 +356,6 @@ public StandardProvenanceEventRecord nextRecord() throws IOException { } builder.setFlowFileEntryDate(flowFileEntryDate); - builder.setLineageIdentifiers(lineageIdentifiers); builder.setLineageStartDate(lineageStartDate); builder.setStorageLocation(filename, startOffset); diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/StandardRecordWriter.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/StandardRecordWriter.java index a5c121abdaf2..35832c475f0d 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/StandardRecordWriter.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/StandardRecordWriter.java @@ -165,8 +165,6 @@ public synchronized long writeRecord(final ProvenanceEventRecord record, final l out.writeLong(record.getEventTime()); out.writeLong(record.getFlowFileEntryDate()); out.writeLong(record.getEventDuration()); - - writeUUIDs(out, record.getLineageIdentifiers()); out.writeLong(record.getLineageStartDate()); writeNullableString(out, record.getComponentId()); diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexingAction.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexingAction.java index 46be391efd9d..ba99058fcf51 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexingAction.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexingAction.java @@ -93,10 +93,6 @@ public void index(final StandardProvenanceEventRecord record, final IndexWriter doc.add(new LongField(SearchableFields.Identifier.getSearchableFieldName(), record.getEventId(), Store.YES)); } - for (final String lineageIdentifier : record.getLineageIdentifiers()) { - addField(doc, SearchableFields.LineageIdentifier, lineageIdentifier, Store.NO); - } - // If it's event is a FORK, or JOIN, add the FlowFileUUID for all child/parent UUIDs. if (record.getEventType() == ProvenanceEventType.FORK || record.getEventType() == ProvenanceEventType.CLONE || record.getEventType() == ProvenanceEventType.REPLAY) { for (final String uuid : record.getChildUuids()) { diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LineageQuery.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LineageQuery.java index 3a2d6e1760f7..27060824546d 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LineageQuery.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LineageQuery.java @@ -47,7 +47,7 @@ public class LineageQuery { private static final Logger logger = LoggerFactory.getLogger(LineageQuery.class); public static Set computeLineageForFlowFiles(final PersistentProvenanceRepository repo, final IndexManager indexManager, final File indexDirectory, - final String lineageIdentifier, final Collection flowFileUuids, final int maxAttributeChars) throws IOException { + final String lineageIdentifier, final Collection flowFileUuids, final int maxAttributeChars) throws IOException { if (requireNonNull(flowFileUuids).size() > MAX_LINEAGE_UUIDS) { throw new IllegalArgumentException(String.format("Cannot compute lineage for more than %s FlowFiles. This lineage contains %s.", MAX_LINEAGE_UUIDS, flowFileUuids.size())); } @@ -73,25 +73,8 @@ public static Set computeLineageForFlowFiles(final Persis flowFileIdQuery.setMinimumNumberShouldMatch(1); } - BooleanQuery query; - if (lineageIdentifier == null) { - query = flowFileIdQuery; - } else { - final BooleanQuery lineageIdQuery = new BooleanQuery(); - lineageIdQuery.add(new TermQuery(new Term(SearchableFields.LineageIdentifier.getSearchableFieldName(), lineageIdentifier)), Occur.MUST); - - if (flowFileIdQuery == null) { - query = lineageIdQuery; - } else { - query = new BooleanQuery(); - query.add(flowFileIdQuery, Occur.SHOULD); - query.add(lineageIdQuery, Occur.SHOULD); - query.setMinimumNumberShouldMatch(1); - } - } - final long searchStart = System.nanoTime(); - final TopDocs uuidQueryTopDocs = searcher.search(query, MAX_QUERY_RESULTS); + final TopDocs uuidQueryTopDocs = searcher.search(flowFileIdQuery, MAX_QUERY_RESULTS); final long searchEnd = System.nanoTime(); // Always authorized. We do this because we need to pull back the event, regardless of whether or not @@ -100,7 +83,7 @@ public static Set computeLineageForFlowFiles(final Persis final DocsReader docsReader = new DocsReader(); final Set recs = docsReader.read(uuidQueryTopDocs, authCheck, searcher.getIndexReader(), repo.getAllLogFiles(), - new AtomicInteger(0), Integer.MAX_VALUE, maxAttributeChars); + new AtomicInteger(0), Integer.MAX_VALUE, maxAttributeChars); final long readDocsEnd = System.nanoTime(); logger.debug("Finished Lineage Query against {}; Lucene search took {} millis, reading records took {} millis", @@ -113,7 +96,7 @@ public static Set computeLineageForFlowFiles(final Persis } catch (final FileNotFoundException fnfe) { // nothing has been indexed yet, or the data has already aged off logger.warn("Attempted to search Provenance Index {} but could not find the file due to {}", indexDirectory, fnfe); - if ( logger.isDebugEnabled() ) { + if (logger.isDebugEnabled()) { logger.warn("", fnfe); } diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestUtil.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestUtil.java index a34d78b8615a..514a43eaf6a3 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestUtil.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestUtil.java @@ -17,9 +17,7 @@ package org.apache.nifi.provenance; import java.util.HashMap; -import java.util.HashSet; import java.util.Map; -import java.util.Set; import org.apache.nifi.flowfile.FlowFile; @@ -38,11 +36,6 @@ public long getEntryDate() { return System.currentTimeMillis(); } - @Override - public Set getLineageIdentifiers() { - return new HashSet(); - } - @Override public long getLineageStartDate() { return System.currentTimeMillis(); diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java index ce53393e82a3..6e7c2f22a724 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java @@ -51,7 +51,6 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutorService; @@ -709,11 +708,6 @@ public long getLineageStartDate() { return record.getLineageStartDate(); } - @Override - public Set getLineageIdentifiers() { - return record.getLineageIdentifiers(); - } - @Override public long getFileSize() { return record.getFileSize(); diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/test/java/org/apache/nifi/provenance/TestVolatileProvenanceRepository.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/test/java/org/apache/nifi/provenance/TestVolatileProvenanceRepository.java index ccc88925ba1c..7db650d99770 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/test/java/org/apache/nifi/provenance/TestVolatileProvenanceRepository.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/test/java/org/apache/nifi/provenance/TestVolatileProvenanceRepository.java @@ -27,10 +27,8 @@ import java.io.IOException; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.UUID; import static org.junit.Assert.assertEquals; @@ -133,11 +131,6 @@ public long getEntryDate() { return System.currentTimeMillis(); } - @Override - public Set getLineageIdentifiers() { - return new HashSet(); - } - @Override public long getLineageStartDate() { return System.currentTimeMillis(); diff --git a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteProvenanceReportingTask.java b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteProvenanceReportingTask.java index 4a897f771640..268b7349570d 100644 --- a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteProvenanceReportingTask.java +++ b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteProvenanceReportingTask.java @@ -249,19 +249,9 @@ static JsonObject serialize(final JsonBuilderFactory factory, final JsonObjectBu addField(builder, "eventOrdinal", event.getEventId()); addField(builder, "eventType", event.getEventType().name()); addField(builder, "timestampMillis", event.getEventTime()); - - addField(builder, "timestamp", df.format(event.getEventTime())); - addField(builder, "durationMillis", event.getEventDuration()); addField(builder, "lineageStart", event.getLineageStartDate()); - - final Set lineageIdentifiers = new HashSet<>(); - if (event.getLineageIdentifiers() != null) { - lineageIdentifiers.addAll(event.getLineageIdentifiers()); - } - lineageIdentifiers.add(event.getFlowFileUuid()); - addField(builder, factory, "lineageIdentifiers", lineageIdentifiers); addField(builder, "details", event.getDetails()); addField(builder, "componentId", event.getComponentId()); addField(builder, "componentType", event.getComponentType()); diff --git a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/test/java/org/apache/nifi/reporting/TestSiteToSiteProvenanceReportingTask.java b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/test/java/org/apache/nifi/reporting/TestSiteToSiteProvenanceReportingTask.java index 265bdd0ed7ee..a048f5b211ce 100644 --- a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/test/java/org/apache/nifi/reporting/TestSiteToSiteProvenanceReportingTask.java +++ b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/test/java/org/apache/nifi/reporting/TestSiteToSiteProvenanceReportingTask.java @@ -23,10 +23,8 @@ import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; @@ -69,10 +67,6 @@ public void testSerializedForm() throws IOException, InitializationException { final Map prevAttrs = new HashMap<>(); attributes.put("filename", "1234.xyz"); - final Set lineageIdentifiers = new HashSet<>(); - lineageIdentifiers.add("123"); - lineageIdentifiers.add("321"); - final ProvenanceEventBuilder builder = new StandardProvenanceEventRecord.Builder(); builder.setEventTime(System.currentTimeMillis()); builder.setEventType(ProvenanceEventType.RECEIVE); @@ -82,7 +76,6 @@ public void testSerializedForm() throws IOException, InitializationException { builder.setAttributes(prevAttrs, attributes); builder.setComponentId("1234"); builder.setComponentType("dummy processor"); - builder.setLineageIdentifiers(lineageIdentifiers); final ProvenanceEventRecord event = builder.build(); final List dataSent = new ArrayList<>(); From 9ad505af68c3cb3cf6a316ed6aa6ba4ba991610b Mon Sep 17 00:00:00 2001 From: joewitt Date: Thu, 14 Jul 2016 00:51:04 -0400 Subject: [PATCH 3/3] NIFI-1157 searched for and resolved all remaining references to deprecated items that were clearly addressable. --- .../PlaceholderProvenanceEvent.java | 1 - .../StandardProvenanceEventRecord.java | 1 - .../processor/util/StandardValidators.java | 52 --- .../org/apache/nifi/util/BooleanHolder.java | 37 -- .../org/apache/nifi/util/IntegerHolder.java | 63 ---- .../java/org/apache/nifi/util/LongHolder.java | 65 ---- .../org/apache/nifi/util/ObjectHolder.java | 44 --- .../org/apache/nifi/util/MockFlowFile.java | 1 - .../util/StandardProcessorTestRunner.java | 17 - .../java/org/apache/nifi/util/TestRunner.java | 7 - .../util/TestStandardProcessorTestRunner.java | 20 - .../nifi/processors/avro/SplitAvro.java | 6 +- .../processors/cassandra/QueryCassandra.java | 6 +- .../controller/FileSystemSwapManager.java | 2 - .../nifi/controller/FlowController.java | 2 - .../controller/StandardFlowFileQueue.java | 1 - .../controller/StandardProcessorNode.java | 1 - .../repository/FileSystemRepository.java | 3 +- .../repository/StandardProcessSession.java | 50 ++- .../io/ByteCountingInputStream.java | 11 +- .../io/ByteCountingOutputStream.java | 11 +- .../controller/repository/io/LongHolder.java | 46 --- .../EventDrivenSchedulingAgent.java | 1 - .../scheduling/StandardProcessScheduler.java | 2 - .../StandardControllerServiceProvider.java | 4 +- .../tasks/ContinuallyRunConnectableTask.java | 1 - .../tasks/ReportingTaskWrapper.java | 1 - .../nifi/groups/StandardProcessGroup.java | 2 - .../controller/TestFileSystemSwapManager.java | 1 - .../controller/TestStandardFlowFileQueue.java | 2 - .../TestWriteAheadFlowFileRepository.java | 2 - .../java/org/apache/nifi/util/FileUtils.java | 71 ---- .../apache/nifi/web/api/dto/DtoFactory.java | 22 +- .../nifi/web/OptimisticLockingManager.java | 57 --- .../nifi/processors/hadoop/ListHDFS.java | 87 +---- .../java/org/apache/nifi/hbase/GetHBase.java | 6 +- .../org/apache/nifi/hbase/PutHBaseJSON.java | 8 +- .../nifi/processors/hive/SelectHiveQL.java | 6 +- .../processors/kite/ConvertAvroSchema.java | 4 +- .../processors/kite/ConvertCSVToAvro.java | 4 +- .../processors/kite/ConvertJSONToAvro.java | 4 +- .../image/ExtractImageMetadata.java | 4 +- .../PersistentProvenanceRepository.java | 1 - .../VolatileProvenanceRepository.java | 3 +- .../SiteToSiteProvenanceReportingTask.java | 2 - .../processors/solr/PutSolrContentStream.java | 6 +- .../nifi/processors/splunk/PutSplunk.java | 4 +- .../standard/AbstractJsonPathProcessor.java | 4 +- .../nifi/processors/standard/BinFiles.java | 354 ------------------ .../processors/standard/CompressContent.java | 4 +- .../processors/standard/ConvertJSONToSQL.java | 4 +- .../processors/standard/EvaluateJsonPath.java | 4 +- .../processors/standard/EvaluateXPath.java | 7 +- .../processors/standard/EvaluateXQuery.java | 6 +- .../nifi/processors/standard/ExecuteSQL.java | 6 +- .../nifi/processors/standard/HashContent.java | 4 +- .../processors/standard/IdentifyMimeType.java | 4 +- .../nifi/processors/standard/JmsConsumer.java | 4 +- .../processors/standard/MergeContent.java | 8 +- .../nifi/processors/standard/PostHTTP.java | 3 +- .../processors/standard/PutFileTransfer.java | 4 +- .../nifi/processors/standard/PutSyslog.java | 4 +- .../standard/QueryDatabaseTable.java | 6 +- .../processors/standard/RouteOnContent.java | 4 +- .../nifi/processors/standard/ScanContent.java | 3 +- .../nifi/processors/standard/SplitText.java | 12 +- .../nifi/processors/standard/SplitXml.java | 4 +- .../nifi/processors/standard/TailFile.java | 4 +- .../processors/standard/UnpackContent.java | 4 +- .../nifi/processors/standard/ValidateXml.java | 4 +- .../standard/servlets/ListenHTTPServlet.java | 3 - .../nifi/processors/standard/util/Bin.java | 170 --------- .../processors/standard/util/BinManager.java | 238 ------------ .../standard/util/FlowFileSessionWrapper.java | 49 --- .../util/crypto/BcryptCipherProvider.java | 21 -- .../util/crypto/NiFiLegacyCipherProvider.java | 17 +- .../crypto/OpenSSLPKCS5CipherProvider.java | 16 - .../util/crypto/PBECipherProvider.java | 11 - .../util/crypto/PBKDF2CipherProvider.java | 17 - .../util/crypto/ScryptCipherProvider.java | 20 - .../BcryptCipherProviderGroovyTest.groovy | 12 +- .../ScryptCipherProviderGroovyTest.groovy | 13 +- .../processors/standard/TestListenSyslog.java | 4 +- .../nifi/dbcp/DatabaseSystemDescriptor.java | 51 --- .../org/apache/nifi/dbcp/DatabaseSystems.java | 83 ---- .../apache/nifi/dbcp/TestDatabaseSystems.java | 31 -- .../org/apache/nifi/ssl/TestProcessor.java | 4 +- 87 files changed, 129 insertions(+), 1844 deletions(-) delete mode 100644 nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/BooleanHolder.java delete mode 100644 nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/IntegerHolder.java delete mode 100644 nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/LongHolder.java delete mode 100644 nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/ObjectHolder.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/LongHolder.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/OptimisticLockingManager.java delete mode 100644 nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/BinFiles.java delete mode 100644 nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/Bin.java delete mode 100644 nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/BinManager.java delete mode 100644 nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FlowFileSessionWrapper.java delete mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DatabaseSystemDescriptor.java delete mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DatabaseSystems.java delete mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/test/java/org/apache/nifi/dbcp/TestDatabaseSystems.java diff --git a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/PlaceholderProvenanceEvent.java b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/PlaceholderProvenanceEvent.java index 083b13e9d792..26696c8b17c8 100644 --- a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/PlaceholderProvenanceEvent.java +++ b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/PlaceholderProvenanceEvent.java @@ -20,7 +20,6 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.Set; /** * A Provenance Event that is used to replace another Provenance Event when authorizations diff --git a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java index 7b0c91f7f844..34de366e12d9 100644 --- a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java +++ b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java @@ -104,7 +104,6 @@ private StandardProvenanceEventRecord(final Builder builder) { updatedAttributes = builder.updatedAttributes == null ? Collections.emptyMap() : Collections.unmodifiableMap(builder.updatedAttributes); sourceQueueIdentifier = builder.sourceQueueIdentifier; - } public String getStorageFilename() { diff --git a/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java b/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java index 2419c897e4c9..47d5d5034e89 100644 --- a/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java +++ b/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java @@ -21,14 +21,12 @@ import java.net.URL; import java.nio.charset.Charset; import java.nio.charset.UnsupportedCharsetException; -import java.util.Collection; import java.util.concurrent.TimeUnit; import java.util.regex.Pattern; import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; import org.apache.nifi.components.Validator; -import org.apache.nifi.controller.ControllerService; import org.apache.nifi.expression.AttributeExpression.ResultType; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.processor.DataUnit; @@ -710,54 +708,4 @@ public ValidationResult validate(final String subject, final String value, final } } - /** - * Creates a validator based on existence of a {@link ControllerService}. - * - * @param serviceClass the controller service API your - * {@link ConfigurableComponent} depends on - * @return a Validator - * @deprecated As of release 0.1.0-incubating, replaced by - * {@link org.apache.nifi.components.PropertyDescriptor.Builder#identifiesControllerService(Class)} - */ - @Deprecated - public static Validator createControllerServiceExistsValidator(final Class serviceClass) { - return new Validator() { - @Override - public ValidationResult validate(final String subject, final String input, final ValidationContext context) { - if (context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input)) { - return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").valid(true).build(); - } - - final ControllerService svc = context.getControllerServiceLookup().getControllerService(input); - - if (svc == null) { - return new ValidationResult.Builder().valid(false).input(input).subject(subject).explanation("No Controller Service exists with this ID").build(); - } - - if (!serviceClass.isAssignableFrom(svc.getClass())) { - return new ValidationResult.Builder() - .valid(false) - .input(input) - .subject(subject) - .explanation("Controller Service with this ID is of type " + svc.getClass().getName() + " but is expected to be of type " + serviceClass.getName()) - .build(); - } - - final ValidationContext serviceValidationContext = context.getControllerServiceValidationContext(svc); - final Collection serviceValidationResults = svc.validate(serviceValidationContext); - for (final ValidationResult result : serviceValidationResults) { - if (!result.isValid()) { - return new ValidationResult.Builder() - .valid(false) - .input(input) - .subject(subject) - .explanation("Controller Service " + input + " is not valid: " + result.getExplanation()) - .build(); - } - } - - return new ValidationResult.Builder().input(input).subject(subject).valid(true).build(); - } - }; - } } diff --git a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/BooleanHolder.java b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/BooleanHolder.java deleted file mode 100644 index 8283389a2a6b..000000000000 --- a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/BooleanHolder.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.util; - -import java.util.concurrent.atomic.AtomicBoolean; - -/** - * @deprecated As of release 1.0.1. Please use {@link AtomicBoolean} - * - * Wraps an Boolean value so that it can be declared final and still be accessed from inner classes; - * the functionality is similar to that of an AtomicBoolean, but operations on this class - * are not atomic. This results in greater performance when the atomicity is not needed. - * - */ - -@Deprecated -public class BooleanHolder extends ObjectHolder { - - public BooleanHolder(final boolean initialValue) { - super(initialValue); - } - -} diff --git a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/IntegerHolder.java b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/IntegerHolder.java deleted file mode 100644 index 8abfdb153f52..000000000000 --- a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/IntegerHolder.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * 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.util; - -import java.util.concurrent.atomic.AtomicInteger; - -/** - * @deprecated As of release 1.0.1. Please use {@link AtomicInteger} - * - * Wraps an Integer value so that it can be declared final and still be accessed from inner classes; - * the functionality is similar to that of an AtomicInteger, but operations on this class - * are not atomic. This results in greater performance when the atomicity is not needed. - * - */ - -@Deprecated -public class IntegerHolder extends ObjectHolder { - - public IntegerHolder(final int initialValue) { - super(initialValue); - } - - public int addAndGet(final int delta) { - final int curValue = get(); - final int newValue = curValue + delta; - set(newValue); - return newValue; - } - - public int getAndAdd(final int delta) { - final int curValue = get(); - final int newValue = curValue + delta; - set(newValue); - return curValue; - } - - public int incrementAndGet() { - return addAndGet(1); - } - - public int getAndIncrement() { - return getAndAdd(1); - } - - public int decrementAndGet() { - return addAndGet(-1); - } - -} diff --git a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/LongHolder.java b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/LongHolder.java deleted file mode 100644 index 723463a13500..000000000000 --- a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/LongHolder.java +++ /dev/null @@ -1,65 +0,0 @@ -/* - * 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.util; - -import java.util.concurrent.atomic.AtomicLong; - -/** - * @deprecated As of release 1.0.1. Please use {@link AtomicLong} - * - * Wraps a Long value so that it can be declared final and still be accessed from inner classes; - * the functionality is similar to that of an AtomicLong, but operations on this class - * are not atomic. This results in greater performance when the atomicity is not needed. - */ - -@Deprecated -public class LongHolder extends ObjectHolder { - - public LongHolder(final long initialValue) { - super(initialValue); - } - - public long addAndGet(final long delta) { - final long curValue = get(); - final long newValue = curValue + delta; - set(newValue); - return newValue; - } - - public long getAndAdd(final long delta) { - final long curValue = get(); - final long newValue = curValue + delta; - set(newValue); - return curValue; - } - - public long incrementAndGet() { - return addAndGet(1); - } - - public long getAndIncrement() { - return getAndAdd(1); - } - - public long decrementAndGet() { - return addAndGet(-1L); - } - - public long getAndDecrement() { - return getAndAdd(-1L); - } -} diff --git a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/ObjectHolder.java b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/ObjectHolder.java deleted file mode 100644 index c577d6f5cd5b..000000000000 --- a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/ObjectHolder.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * 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.util; - -import java.util.concurrent.atomic.AtomicReference; - -/** - * @deprecated As of release 0.7.0. Please use {@link AtomicReference} - * - * A bean that holds a single value of type T. - * - */ - -@Deprecated -public class ObjectHolder { - - private T value; - - public ObjectHolder(final T initialValue) { - this.value = initialValue; - } - - public T get() { - return value; - } - - public void set(T value) { - this.value = value; - } -} diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockFlowFile.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockFlowFile.java index 02806b0e8dfc..516c8a44b7de 100644 --- a/nifi-mock/src/main/java/org/apache/nifi/util/MockFlowFile.java +++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockFlowFile.java @@ -28,7 +28,6 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.Map; import java.util.Set; import java.util.UUID; diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java b/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java index 7d49f0f236e8..b80c09f1d7e9 100644 --- a/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java +++ b/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java @@ -65,7 +65,6 @@ import org.apache.nifi.processor.Processor; import org.apache.nifi.processor.Relationship; import org.apache.nifi.provenance.ProvenanceEventRecord; -import org.apache.nifi.provenance.ProvenanceReporter; import org.apache.nifi.reporting.InitializationException; import org.apache.nifi.state.MockStateManager; import org.junit.Assert; @@ -459,15 +458,6 @@ public int compare(final MockFlowFile o1, final MockFlowFile o2) { return flowFiles; } - /** - * @deprecated The ProvenanceReporter should not be accessed through the test runner, as it does not expose the events that were emitted. - */ - @Override - @Deprecated - public ProvenanceReporter getProvenanceReporter() { - return sharedState.getProvenanceReporter(); - } - @Override public QueueSize getQueueSize() { return flowFileQueue.size(); @@ -584,13 +574,6 @@ public void addControllerService(final String identifier, final ControllerServic @Override public void addControllerService(final String identifier, final ControllerService service, final Map properties) throws InitializationException { - // hold off on failing due to deprecated annotation for now... will introduce later. - // for ( final Method method : service.getClass().getMethods() ) { - // if ( method.isAnnotationPresent(org.apache.nifi.controller.annotation.OnConfigured.class) ) { - // Assert.fail("Controller Service " + service + " is using deprecated Annotation " + org.apache.nifi.controller.annotation.OnConfigured.class + " for method " + method); - // } - // } - final MockComponentLog logger = new MockComponentLog(identifier, service); controllerServiceLoggers.put(identifier, logger); final MockStateManager serviceStateManager = new MockStateManager(service); diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java b/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java index c3bedb1915c4..44d7f9fb9cd2 100644 --- a/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java +++ b/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java @@ -33,7 +33,6 @@ import org.apache.nifi.processor.Processor; import org.apache.nifi.processor.Relationship; import org.apache.nifi.provenance.ProvenanceEventRecord; -import org.apache.nifi.provenance.ProvenanceReporter; import org.apache.nifi.reporting.InitializationException; import org.apache.nifi.state.MockStateManager; @@ -485,12 +484,6 @@ public interface TestRunner { */ List getPenalizedFlowFiles(); - /** - * @return the {@link ProvenanceReporter} that will be used by the - * configured {@link Processor} for reporting Provenance Events - */ - ProvenanceReporter getProvenanceReporter(); - /** * @return the current size of the Processor's Input Queue */ diff --git a/nifi-mock/src/test/java/org/apache/nifi/util/TestStandardProcessorTestRunner.java b/nifi-mock/src/test/java/org/apache/nifi/util/TestStandardProcessorTestRunner.java index 0e41e89f26d3..f5b28aaf87c9 100644 --- a/nifi-mock/src/test/java/org/apache/nifi/util/TestStandardProcessorTestRunner.java +++ b/nifi-mock/src/test/java/org/apache/nifi/util/TestStandardProcessorTestRunner.java @@ -110,26 +110,6 @@ public void testAllFlowFilesContainAttribute() { runner.assertAllFlowFilesContainAttribute(AddAttributeProcessor.KEY); } - @org.apache.nifi.annotation.documentation.Tags({"deprecated"}) - private static class NewAnnotation extends AbstractProcessor { - - @Override - public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { - } - } - - private static class NewMethodAnnotation extends AbstractProcessor { - - @org.apache.nifi.annotation.lifecycle.OnScheduled - public void dummy() { - - } - - @Override - public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { - } - } - private static class ProcessorWithOnStop extends AbstractProcessor { private int callsWithContext = 0; diff --git a/nifi-nar-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/SplitAvro.java b/nifi-nar-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/SplitAvro.java index 38e3a0dfc058..ac6936f1d420 100644 --- a/nifi-nar-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/SplitAvro.java +++ b/nifi-nar-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/SplitAvro.java @@ -27,6 +27,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; import org.apache.avro.file.CodecFactory; import org.apache.avro.file.DataFileConstants; @@ -57,7 +58,6 @@ import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.stream.io.BufferedOutputStream; -import org.apache.nifi.util.ObjectHolder; @SideEffectFree @SupportsBatching @@ -231,7 +231,7 @@ public RecordSplitter(final int splitSize, final boolean transferMetadata) { @Override public List split(final ProcessSession session, final FlowFile originalFlowFile, final SplitWriter splitWriter) { final List childFlowFiles = new ArrayList<>(); - final ObjectHolder recordHolder = new ObjectHolder<>(null); + final AtomicReference recordHolder = new AtomicReference<>(null); session.read(originalFlowFile, new InputStreamCallback() { @Override @@ -239,7 +239,7 @@ public void process(InputStream rawIn) throws IOException { try (final InputStream in = new BufferedInputStream(rawIn); final DataFileStream reader = new DataFileStream<>(in, new GenericDatumReader())) { - final ObjectHolder codec = new ObjectHolder<>(reader.getMetaString(DataFileConstants.CODEC)); + final AtomicReference codec = new AtomicReference<>(reader.getMetaString(DataFileConstants.CODEC)); if (codec.get() == null) { codec.set(DataFileConstants.NULL_CODEC); } diff --git a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/QueryCassandra.java b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/QueryCassandra.java index 318295826038..ca7f690c0300 100644 --- a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/QueryCassandra.java +++ b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/QueryCassandra.java @@ -52,7 +52,6 @@ import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; -import org.apache.nifi.util.LongHolder; import org.apache.nifi.util.StopWatch; import java.io.IOException; @@ -71,6 +70,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicLong; @Tags({"cassandra", "cql", "select"}) @EventDriven @@ -228,7 +228,7 @@ public void onTrigger(final ProcessContext context, final ProcessSession session // and states that it is thread-safe. This is why connectionSession is not in a try-with-resources. final Session connectionSession = cassandraSession.get(); final ResultSetFuture queryFuture = connectionSession.executeAsync(selectQuery); - final LongHolder nrOfRows = new LongHolder(0L); + final AtomicLong nrOfRows = new AtomicLong(0L); fileToProcess = session.write(fileToProcess, new OutputStreamCallback() { @Override @@ -259,7 +259,7 @@ public void process(final OutputStream out) throws IOException { }); // set attribute how many rows were selected - fileToProcess = session.putAttribute(fileToProcess, RESULT_ROW_COUNT, nrOfRows.get().toString()); + fileToProcess = session.putAttribute(fileToProcess, RESULT_ROW_COUNT, String.valueOf(nrOfRows.get())); logger.info("{} contains {} Avro records; transferring to 'success'", new Object[]{fileToProcess, nrOfRows.get()}); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FileSystemSwapManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FileSystemSwapManager.java index 5a33eecbfdfd..a4c267c1db34 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FileSystemSwapManager.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FileSystemSwapManager.java @@ -284,8 +284,6 @@ public SwapSummary getSwapSummary(final String swapLocation) throws IOException } } - - @SuppressWarnings("deprecation") public static int serializeFlowFiles(final List toSwap, final FlowFileQueue queue, final String swapLocation, final OutputStream destination) throws IOException { if (toSwap == null || toSwap.isEmpty()) { return 0; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java index 76b946ffb7da..80b89dd4f455 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java @@ -986,7 +986,6 @@ public ProcessorNode createProcessor(final String type, final String id) throws * @throws NullPointerException if either arg is null * @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(); @@ -3605,7 +3604,6 @@ public ProvenanceEventRecord replayFlowFile(final long provenanceEventRecordId, return replayFlowFile(record, user); } - @SuppressWarnings("deprecation") public ProvenanceEventRecord replayFlowFile(final ProvenanceEventRecord event, final NiFiUser user) throws IOException { if (event == null) { throw new NullPointerException(); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java index aa77765f92c6..f391da555197 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java @@ -733,7 +733,6 @@ private Prioritizer(final List priorities) { } @Override - @SuppressWarnings("deprecation") public int compare(final FlowFileRecord f1, final FlowFileRecord f2) { int returnVal = 0; final boolean f1Penalized = f1.isPenalized(); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java index b232f265388a..9d2c45cf0249 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java @@ -139,7 +139,6 @@ public StandardProcessorNode(final Processor processor, final String uuid, processor.getClass().getSimpleName(), processor.getClass().getCanonicalName()); } - @SuppressWarnings("deprecation") public StandardProcessorNode(final Processor processor, final String uuid, final ValidationContextFactory validationContextFactory, final ProcessScheduler scheduler, final ControllerServiceProvider controllerServiceProvider, diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java index 210fcca57c42..673440fa6e25 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java @@ -70,7 +70,6 @@ import org.apache.nifi.stream.io.StreamUtils; import org.apache.nifi.stream.io.SynchronizedByteCountingOutputStream; import org.apache.nifi.util.FormatUtils; -import org.apache.nifi.util.LongHolder; import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.util.StopWatch; import org.apache.nifi.util.file.FileUtils; @@ -297,7 +296,7 @@ private synchronized void initializeRepository() throws IOException { final Callable scanContainer = new Callable() { @Override public Long call() throws IOException { - final LongHolder oldestDateHolder = new LongHolder(0L); + final AtomicLong oldestDateHolder = new AtomicLong(0L); // the path already exists, so scan the path to find any files and update maxIndex to the max of // all filenames seen. diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java index 1e2d63491934..0a2f8c9e7a44 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java @@ -54,7 +54,6 @@ import org.apache.nifi.controller.repository.io.FlowFileAccessInputStream; import org.apache.nifi.controller.repository.io.FlowFileAccessOutputStream; import org.apache.nifi.controller.repository.io.LimitedInputStream; -import org.apache.nifi.controller.repository.io.LongHolder; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; import org.apache.nifi.processor.FlowFileFilter; @@ -120,8 +119,8 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE private int removedCount = 0; // number of flowfiles removed in this session private long removedBytes = 0L; // size of all flowfiles removed in this session - private final LongHolder bytesRead = new LongHolder(0L); - private final LongHolder bytesWritten = new LongHolder(0L); + private final AtomicLong bytesRead = new AtomicLong(0L); + private final AtomicLong bytesWritten = new AtomicLong(0L); private int flowFilesIn = 0, flowFilesOut = 0; private long contentSizeIn = 0L, contentSizeOut = 0L; @@ -966,8 +965,8 @@ private void rollback(final boolean penalize, final boolean rollbackCheckpoint) final Connectable connectable = context.getConnectable(); final StandardFlowFileEvent flowFileEvent = new StandardFlowFileEvent(connectable.getIdentifier()); - flowFileEvent.setBytesRead(bytesRead.getValue()); - flowFileEvent.setBytesWritten(bytesWritten.getValue()); + flowFileEvent.setBytesRead(bytesRead.get()); + flowFileEvent.setBytesWritten(bytesWritten.get()); // update event repository try { @@ -1055,8 +1054,8 @@ private void resetState() { flowFilesOut = 0; removedCount = 0; removedBytes = 0L; - bytesRead.setValue(0L); - bytesWritten.setValue(0L); + bytesRead.set(0L); + bytesWritten.set(0L); connectionCounts.clear(); createdFlowFiles.clear(); removedFlowFiles.clear(); @@ -1822,7 +1821,7 @@ private InputStream getInputStream(final FlowFile flowFile, final ContentClaim c } currentReadClaim = claim; - currentReadClaimStream = new ByteCountingInputStream(rawInStream, new LongHolder(0L)); + currentReadClaimStream = new ByteCountingInputStream(rawInStream, new AtomicLong(0L)); StreamUtils.skip(currentReadClaimStream, offset); // Use a non-closeable stream because we want to keep it open after the callback has finished so that we can @@ -2069,8 +2068,8 @@ public FlowFile merge(final Collection sources, final FlowFile destina writtenCount += footer.length; } } finally { - bytesWritten.increment(writtenCount); - bytesRead.increment(readCount); + bytesWritten.getAndAdd(writtenCount); + bytesRead.getAndAdd(readCount); } } catch (final ContentNotFoundException nfe) { destroyContent(newClaim); @@ -2111,7 +2110,7 @@ public FlowFile write(final FlowFile source, final OutputStreamCallback writer) final StandardRepositoryRecord record = records.get(source); ContentClaim newClaim = null; - final LongHolder writtenHolder = new LongHolder(0L); + final AtomicLong writtenHolder = new AtomicLong(0L); try { newClaim = context.getContentRepository().create(context.getConnectable().isLossTolerant()); claimLog.debug("Creating ContentClaim {} for 'write' for {}", newClaim, source); @@ -2142,7 +2141,7 @@ public FlowFile write(final FlowFile source, final OutputStreamCallback writer) destroyContent(newClaim); throw t; } finally { - bytesWritten.increment(writtenHolder.getValue()); + bytesWritten.getAndAdd(writtenHolder.get()); } removeTemporaryClaim(record); @@ -2150,7 +2149,7 @@ public FlowFile write(final FlowFile source, final OutputStreamCallback writer) .fromFlowFile(record.getCurrent()) .contentClaim(newClaim) .contentClaimOffset(0) - .size(writtenHolder.getValue()) + .size(writtenHolder.get()) .build(); record.setWorking(newFile); @@ -2178,7 +2177,7 @@ public FlowFile append(final FlowFile source, final OutputStreamCallback writer) final OutputStream rawOutStream = context.getContentRepository().write(newClaim); final OutputStream bufferedOutStream = new BufferedOutputStream(rawOutStream); - outStream = new ByteCountingOutputStream(bufferedOutStream, new LongHolder(0L)); + outStream = new ByteCountingOutputStream(bufferedOutStream, new AtomicLong(0L)); originalByteWrittenCount = 0; appendableStreams.put(newClaim, outStream); @@ -2224,7 +2223,7 @@ public FlowFile append(final FlowFile source, final OutputStreamCallback writer) } finally { if (outStream != null) { final long bytesWrittenThisIteration = outStream.getBytesWritten() - originalByteWrittenCount; - bytesWritten.increment(bytesWrittenThisIteration); + bytesWritten.getAndAdd(bytesWrittenThisIteration); } } @@ -2313,7 +2312,7 @@ public FlowFile write(final FlowFile source, final StreamCallback writer) { final ContentClaim currClaim = record.getCurrentClaim(); ContentClaim newClaim = null; - final LongHolder writtenHolder = new LongHolder(0L); + final AtomicLong writtenHolder = new AtomicLong(0L); try { newClaim = context.getContentRepository().create(context.getConnectable().isLossTolerant()); claimLog.debug("Creating ContentClaim {} for 'write' for {}", newClaim, source); @@ -2365,7 +2364,7 @@ public FlowFile write(final FlowFile source, final StreamCallback writer) { destroyContent(newClaim); throw t; } finally { - bytesWritten.increment(writtenHolder.getValue()); + bytesWritten.getAndAdd(writtenHolder.get()); } removeTemporaryClaim(record); @@ -2373,7 +2372,7 @@ public FlowFile write(final FlowFile source, final StreamCallback writer) { .fromFlowFile(record.getCurrent()) .contentClaim(newClaim) .contentClaimOffset(0L) - .size(writtenHolder.getValue()) + .size(writtenHolder.get()) .build(); record.setWorking(newFile); @@ -2405,8 +2404,8 @@ public FlowFile importFrom(final Path source, final boolean keepSourceFile, fina long newSize = 0L; try { newSize = context.getContentRepository().importFrom(source, newClaim); - bytesWritten.increment(newSize); - bytesRead.increment(newSize); + bytesWritten.getAndAdd(newSize); + bytesRead.getAndAdd(newSize); } catch (final Throwable t) { destroyContent(newClaim); throw new FlowFileAccessException("Failed to import data from " + source + " for " + destination + " due to " + t.toString(), t); @@ -2439,7 +2438,7 @@ public FlowFile importFrom(final InputStream source, final FlowFile destination) claimLog.debug("Creating ContentClaim {} for 'importFrom' for {}", newClaim, destination); newSize = context.getContentRepository().importFrom(source, newClaim); - bytesWritten.increment(newSize); + bytesWritten.getAndAdd(newSize); } catch (final IOException e) { throw new FlowFileAccessException("Unable to create ContentClaim due to " + e.toString(), e); } @@ -2465,8 +2464,8 @@ public void exportTo(final FlowFile source, final Path destination, final boolea ensureNotAppending(record.getCurrentClaim()); final long copyCount = context.getContentRepository().exportTo(record.getCurrentClaim(), destination, append, record.getCurrentClaimOffset(), source.getSize()); - bytesRead.increment(copyCount); - bytesWritten.increment(copyCount); + bytesRead.getAndAdd(copyCount); + bytesWritten.getAndAdd(copyCount); } catch (final ContentNotFoundException nfe) { handleContentNotFound(nfe, record); } catch (final Throwable t) { @@ -2640,7 +2639,6 @@ public FlowFile create(final Collection parents) { // When creating a new FlowFile from multiple parents, we need to add all of the Lineage Identifiers // and use the earliest lineage start date long lineageStartDate = 0L; - final Set lineageIdentifiers = new HashSet<>(); for (final FlowFile parent : parents) { final long parentLineageStartDate = parent.getLineageStartDate(); @@ -2785,8 +2783,8 @@ private void checkpoint(final StandardProcessSession session, final List= 0) { - bytesReadHolder.increment(1); + bytesReadHolder.getAndIncrement(); } return fromSuper; } @@ -43,7 +44,7 @@ public int read() throws IOException { public int read(byte[] b, int off, int len) throws IOException { final int fromSuper = in.read(b, off, len); if (fromSuper >= 0) { - bytesReadHolder.increment(fromSuper); + bytesReadHolder.getAndAdd(fromSuper); } return fromSuper; @@ -87,7 +88,7 @@ public void close() throws IOException { } public long getBytesRead() { - return bytesReadHolder.getValue(); + return bytesReadHolder.get(); } public long getBytesSkipped() { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/ByteCountingOutputStream.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/ByteCountingOutputStream.java index 4e727e9a2301..7c778a28b992 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/ByteCountingOutputStream.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/ByteCountingOutputStream.java @@ -18,13 +18,14 @@ import java.io.IOException; import java.io.OutputStream; +import java.util.concurrent.atomic.AtomicLong; public class ByteCountingOutputStream extends OutputStream { - private final LongHolder bytesWrittenHolder; + private final AtomicLong bytesWrittenHolder; private final OutputStream out; - public ByteCountingOutputStream(final OutputStream out, final LongHolder longHolder) { + public ByteCountingOutputStream(final OutputStream out, final AtomicLong longHolder) { this.out = out; this.bytesWrittenHolder = longHolder; } @@ -32,7 +33,7 @@ public ByteCountingOutputStream(final OutputStream out, final LongHolder longHol @Override public void write(int b) throws IOException { out.write(b); - bytesWrittenHolder.increment(1); + bytesWrittenHolder.getAndIncrement(); } @Override @@ -43,11 +44,11 @@ public void write(byte[] b) throws IOException { @Override public void write(byte[] b, int off, int len) throws IOException { out.write(b, off, len); - bytesWrittenHolder.increment(len); + bytesWrittenHolder.getAndAdd(len); } public long getBytesWritten() { - return bytesWrittenHolder.getValue(); + return bytesWrittenHolder.get(); } @Override diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/LongHolder.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/LongHolder.java deleted file mode 100644 index bd5fec13fd69..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/LongHolder.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.controller.repository.io; - -/** - * Class to hold a long value that can be incremented and decremented. This allows the abstraction of passing a long value by reference, rather than by value, without the overhead of synchronization - * required by the use of an AtomicLong. - */ -public class LongHolder { - - private long value; - - public LongHolder() { - value = 0L; - } - - public LongHolder(final long initialValue) { - value = initialValue; - } - - public void increment(long value) { - this.value += value; - } - - public long getValue() { - return value; - } - - public void setValue(final long value) { - this.value = value; - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java index d3410cca50f5..262ac77c81c1 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java @@ -270,7 +270,6 @@ public void run() { } } - @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) { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java index 1729d4ac656c..53fc72635138 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java @@ -181,7 +181,6 @@ public void schedule(final ReportingTaskNode taskNode) { scheduleState.setScheduled(true); final Runnable startReportingTaskRunnable = new Runnable() { - @SuppressWarnings("deprecation") @Override public void run() { final long lastStopTime = scheduleState.getLastStopTime(); @@ -243,7 +242,6 @@ public void unschedule(final ReportingTaskNode taskNode) { taskNode.setScheduledState(ScheduledState.STOPPED); final Runnable unscheduleReportingTaskRunnable = new Runnable() { - @SuppressWarnings("deprecation") @Override public void run() { final ConfigurationContext configurationContext = taskNode.getConfigurationContext(); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java index 22ee5cfd139c..2c8d258eec49 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java @@ -30,6 +30,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; import org.apache.commons.lang3.StringUtils; import org.apache.nifi.annotation.lifecycle.OnAdded; @@ -56,7 +57,6 @@ import org.apache.nifi.processor.StandardValidationContextFactory; import org.apache.nifi.reporting.BulletinRepository; import org.apache.nifi.reporting.Severity; -import org.apache.nifi.util.ObjectHolder; import org.apache.nifi.util.ReflectionUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -144,7 +144,7 @@ public ControllerServiceNode createControllerService(final String type, final St final Class controllerServiceClass = rawClass.asSubclass(ControllerService.class); final ControllerService originalService = controllerServiceClass.newInstance(); - final ObjectHolder serviceNodeHolder = new ObjectHolder<>(null); + final AtomicReference serviceNodeHolder = new AtomicReference<>(null); final InvocationHandler invocationHandler = new InvocationHandler() { @Override public Object invoke(final Object proxy, final Method method, final Object[] args) throws Throwable { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java index 8bc7d994a3aa..8f1623ed31f5 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java @@ -54,7 +54,6 @@ public ContinuallyRunConnectableTask(final ProcessContextFactory contextFactory, } @Override - @SuppressWarnings("deprecation") public Boolean call() { if (!scheduleState.isScheduled()) { return false; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java index 1dc08c2b2f6b..eb087a433171 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java @@ -34,7 +34,6 @@ public ReportingTaskWrapper(final ReportingTaskNode taskNode, final ScheduleStat this.scheduleState = scheduleState; } - @SuppressWarnings("deprecation") @Override public synchronized void run() { scheduleState.incrementActiveThreadCount(); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java index c8d9d2523458..8017abdff859 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java @@ -333,7 +333,6 @@ private StateManager getStateManager(final String componentId) { return flowController.getStateManagerProvider().getStateManager(componentId); } - @SuppressWarnings("deprecation") private void shutdown(final ProcessGroup procGroup) { for (final ProcessorNode node : procGroup.getProcessors()) { try (final NarCloseable x = NarCloseable.withNarLoader()) { @@ -688,7 +687,6 @@ public void addProcessor(final ProcessorNode processor) { } @Override - @SuppressWarnings("deprecation") public void removeProcessor(final ProcessorNode processor) { final String id = requireNonNull(processor).getIdentifier(); writeLock.lock(); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFileSystemSwapManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFileSystemSwapManager.java index 50bc874797a2..4a590f210ba3 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFileSystemSwapManager.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFileSystemSwapManager.java @@ -69,7 +69,6 @@ public void testBackwardCompatible() throws IOException { } @Test - @SuppressWarnings("deprecation") public void testRoundTripSerializeDeserialize() throws IOException { final List toSwap = new ArrayList<>(10000); final Map attrs = new HashMap<>(); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardFlowFileQueue.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardFlowFileQueue.java index 6e368de16bed..32c1dc6dbf97 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardFlowFileQueue.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardFlowFileQueue.java @@ -564,7 +564,6 @@ public List recoverSwapLocations(FlowFileQueue flowFileQueue) throws IOE } @Override - @SuppressWarnings("deprecation") public SwapSummary getSwapSummary(String swapLocation) throws IOException { final List flowFiles = swappedOut.get(swapLocation); if (flowFiles == null) { @@ -668,7 +667,6 @@ public Map getAttributes() { } @Override - @SuppressWarnings("deprecation") public int compareTo(final FlowFile o) { return Long.compare(id, o.getId()); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java index 4094ca4b2d6a..55b742601675 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java @@ -157,7 +157,6 @@ public void testResourceClaimsIncremented() throws IOException { } @Test - @SuppressWarnings("deprecation") public void testRestartWithOneRecord() throws IOException { final Path path = Paths.get("target/test-repo"); if (Files.exists(path)) { @@ -305,7 +304,6 @@ public List recoverSwapLocations(FlowFileQueue flowFileQueue) throws IOE } @Override - @SuppressWarnings("deprecation") public SwapSummary getSwapSummary(String swapLocation) throws IOException { List records = null; for (final Map> swapMap : swappedRecords.values()) { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/util/FileUtils.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/util/FileUtils.java index c02b83f044b7..a9943b8d3da8 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/util/FileUtils.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/util/FileUtils.java @@ -95,77 +95,6 @@ public static boolean deleteFile(final File file, final Logger logger, final int return isGone; } - /** - * Deletes all files (not directories..) in the given directory (non - * recursive) that match the given filename filter. If any file cannot be - * deleted then this is printed at warn to the given logger. - * - * @param directory to delete contents of - * @param filter if null then no filter is used - * @param logger to notify - * @deprecated As of release 0.6.0, replaced by - * {@link #deleteFilesInDirectory(File, FilenameFilter, Logger)} - */ - @Deprecated - public static void deleteFilesInDir(final File directory, final FilenameFilter filter, final Logger logger) { - FileUtils.deleteFilesInDir(directory, filter, logger, false); - } - - /** - * Deletes all files (not directories) in the given directory (recursive) - * that match the given filename filter. If any file cannot be deleted then - * this is printed at warn to the given logger. - * - * @param directory to delete contents of - * @param filter if null then no filter is used - * @param logger to notify - * @param recurse true if should recurse - * @deprecated As of release 0.6.0, replaced by - * {@link #deleteFilesInDirectory(File, FilenameFilter, Logger, boolean)} - */ - @Deprecated - public static void deleteFilesInDir(final File directory, final FilenameFilter filter, final Logger logger, final boolean recurse) { - FileUtils.deleteFilesInDir(directory, filter, logger, recurse, false); - } - - /** - * Deletes all files (not directories) in the given directory (recursive) - * that match the given filename filter. If any file cannot be deleted then - * this is printed at warn to the given logger. - * - * @param directory to delete contents of - * @param filter if null then no filter is used - * @param logger to notify - * @param recurse will look for contents of sub directories. - * @param deleteEmptyDirectories default is false; if true will delete - * directories found that are empty - * @deprecated As of release 0.6.0, replaced by - * {@link #deleteFilesInDirectory(File, FilenameFilter, Logger, boolean, boolean)} - */ - @Deprecated - public static void deleteFilesInDir(final File directory, final FilenameFilter filter, final Logger logger, final boolean recurse, final boolean deleteEmptyDirectories) { - // ensure the specified directory is actually a directory and that it exists - if (null != directory && directory.isDirectory()) { - final File ingestFiles[] = directory.listFiles(); - if (ingestFiles == null) { - // null if abstract pathname does not denote a directory, or if an I/O error occurs - logger.error("Unable to list directory content in: " + directory.getAbsolutePath()); - } - for (File ingestFile : ingestFiles) { - boolean process = (filter == null) ? true : filter.accept(directory, ingestFile.getName()); - if (ingestFile.isFile() && process) { - FileUtils.deleteFile(ingestFile, logger, 3); - } - if (ingestFile.isDirectory() && recurse) { - FileUtils.deleteFilesInDir(ingestFile, filter, logger, recurse, deleteEmptyDirectories); - if (deleteEmptyDirectories && ingestFile.list().length == 0) { - FileUtils.deleteFile(ingestFile, logger, 3); - } - } - } - } - } - /** * Deletes all files (not directories..) in the given directory (non * recursive) that match the given filename filter. If any file cannot be diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java index 7396a3421419..e22ba4bd0ba6 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java @@ -1927,23 +1927,14 @@ private FlowSnippetDTO createProcessGroupContentsDto(final ProcessGroup group, f /** * Gets the capability description from the specified class. */ - @SuppressWarnings("deprecation") private String getCapabilityDescription(final Class cls) { 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(); + return capabilityDesc == null ? null : capabilityDesc.value(); } /** * Gets the tags from the specified class. */ - @SuppressWarnings("deprecation") private Set getTags(final Class cls) { final Set tags = new HashSet<>(); final Tags tagsAnnotation = cls.getAnnotation(Tags.class); @@ -1951,13 +1942,6 @@ private Set getTags(final Class cls) { 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; @@ -2129,7 +2113,6 @@ public BulletinDTO createBulletinDto(final Bulletin bulletin) { * @param node node * @return dto */ - @SuppressWarnings("deprecation") public ProvenanceNodeDTO createProvenanceEventNodeDTO(final ProvenanceEventLineageNode node) { final ProvenanceNodeDTO dto = new ProvenanceNodeDTO(); dto.setId(node.getIdentifier()); @@ -2140,7 +2123,6 @@ public ProvenanceNodeDTO createProvenanceEventNodeDTO(final ProvenanceEventLinea dto.setFlowFileUuid(node.getFlowFileUuid()); dto.setParentUuids(node.getParentUuids()); dto.setChildUuids(node.getChildUuids()); - dto.setClusterNodeIdentifier(node.getClusterNodeIdentifier()); return dto; } @@ -2150,7 +2132,6 @@ public ProvenanceNodeDTO createProvenanceEventNodeDTO(final ProvenanceEventLinea * @param node node * @return dto */ - @SuppressWarnings("deprecation") public ProvenanceNodeDTO createFlowFileNodeDTO(final LineageNode node) { final ProvenanceNodeDTO dto = new ProvenanceNodeDTO(); dto.setId(node.getIdentifier()); @@ -2158,7 +2139,6 @@ public ProvenanceNodeDTO createFlowFileNodeDTO(final LineageNode node) { dto.setTimestamp(new Date(node.getTimestamp())); dto.setMillis(node.getTimestamp()); dto.setFlowFileUuid(node.getFlowFileUuid()); - dto.setClusterNodeIdentifier(node.getClusterNodeIdentifier()); return dto; } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/OptimisticLockingManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/OptimisticLockingManager.java deleted file mode 100644 index b205590497a6..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/OptimisticLockingManager.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.web; - -import org.apache.nifi.web.revision.RevisionManager; - -/** - * A manager for optimistic locking based on revisions. A revision is composed - * of a client ID and a version number. Two revisions are considered equal if - * either their version numbers match or their client IDs match. - * - * @deprecated This class has been deprecated in favor of {@link RevisionManager} - */ -@Deprecated -public interface OptimisticLockingManager { - - /** - * Attempts to execute the specified configuration request using the - * specified revision within a lock. - * - * @param type of snapshot - * @param revision revision - * @param configurationRequest request - * @return snapshot - */ - ConfigurationSnapshot configureFlow(Revision revision, ConfigurationRequest configurationRequest); - - /** - * Updates the revision using the specified revision within a lock. - * - * @param updateRevision new revision - */ - void setRevision(UpdateRevision updateRevision); - - /** - * Returns the last flow modification. This is a combination of the revision - * and the user who performed the modification. - * - * @return the last modification - */ - FlowModification getLastModification(); - -} diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/ListHDFS.java b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/ListHDFS.java index f5daef277224..4cb8d25ef127 100644 --- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/ListHDFS.java +++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/ListHDFS.java @@ -31,10 +31,8 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.state.Scope; -import org.apache.nifi.components.state.StateManager; import org.apache.nifi.components.state.StateMap; import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient; import org.apache.nifi.flowfile.FlowFile; @@ -46,8 +44,6 @@ import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.processors.hadoop.util.HDFSListing; -import org.apache.nifi.processors.hadoop.util.HDFSListing.StateKeys; -import org.apache.nifi.processors.hadoop.util.StringSerDe; import org.codehaus.jackson.JsonNode; import org.codehaus.jackson.JsonParseException; import org.codehaus.jackson.map.JsonMappingException; @@ -124,7 +120,6 @@ public class ListHDFS extends AbstractHadoopProcessor { private volatile long latestTimestampListed = -1L; private volatile long latestTimestampEmitted = -1L; - private volatile boolean electedPrimaryNodeSinceLastIteration = false; private volatile long lastRunTimestamp = -1L; static final String LISTING_TIMESTAMP_KEY = "listing.timestamp"; @@ -175,80 +170,6 @@ private HDFSListing deserialize(final String serializedState) throws JsonParseEx return mapper.readValue(jsonNode, HDFSListing.class); } - /** - * Transitions state from the Distributed cache service to the state manager. This will be - * removed in NiFi 1.x - * - * @param context the ProcessContext - * @throws IOException if unable to communicate with state manager or controller service - */ - @Deprecated - @OnScheduled - public void moveStateToStateManager(final ProcessContext context) throws IOException { - final StateManager stateManager = context.getStateManager(); - final StateMap stateMap = stateManager.getState(Scope.CLUSTER); - - // Check if we have already stored state in the cluster state manager. - if (stateMap.getVersion() == -1L) { - final HDFSListing serviceListing = getListingFromService(context); - if (serviceListing != null) { - context.getStateManager().setState(serviceListing.toMap(), Scope.CLUSTER); - } - } - } - - @Deprecated - private HDFSListing getListingFromService(final ProcessContext context) throws IOException { - final DistributedMapCacheClient client = context.getProperty(DISTRIBUTED_CACHE_SERVICE).asControllerService(DistributedMapCacheClient.class); - if (client == null) { - return null; - } - - final String directory = context.getProperty(DIRECTORY).getValue(); - final String remoteValue = client.get(getKey(directory), new StringSerDe(), new StringSerDe()); - if (remoteValue == null) { - return null; - } - - try { - return deserialize(remoteValue); - } catch (final Exception e) { - getLogger().error("Failed to retrieve state from Distributed Map Cache because the content that was retrieved could not be understood", e); - return null; - } - } - - /** - * Restores state information from the 'old' style of storing state. This is deprecated and will no longer be supported - * in the 1.x NiFi baseline - * - * @param directory the directory that the listing was performed against - * @param remoteListing the remote listing - * @return the minimum timestamp that should be used for new entries - */ - @Deprecated - private Long restoreTimestampFromOldStateFormat(final String directory, final HDFSListing remoteListing) { - // No cluster-wide state has been recovered. Just use whatever values we already have. - if (remoteListing == null) { - return latestTimestampListed; - } - - // If our local timestamp is already later than the remote listing's timestamp, use our local info. - Long minTimestamp = latestTimestampListed; - if (minTimestamp != null && minTimestamp > remoteListing.getLatestTimestamp().getTime()) { - return minTimestamp; - } - - // Use the remote listing's information. - if (minTimestamp == null || electedPrimaryNodeSinceLastIteration) { - this.latestTimestampListed = remoteListing.getLatestTimestamp().getTime(); - this.latestTimestampEmitted = this.latestTimestampListed; - } - - return minTimestamp; - } - - /** * Determines which of the given FileStatus's describes a File that should be listed. * @@ -339,13 +260,7 @@ public void onTrigger(final ProcessContext context, final ProcessSession session } else { // Determine if state is stored in the 'new' format or the 'old' format final String emittedString = stateMap.get(EMITTED_TIMESTAMP_KEY); - if (emittedString == null && stateMap.get(StateKeys.TIMESTAMP) != null) { - // state is stored in the old format with XML - final Map stateValues = stateMap.toMap(); - final HDFSListing stateListing = HDFSListing.fromMap(stateValues); - getLogger().debug("Found old-style state stored"); - restoreTimestampFromOldStateFormat(directory, stateListing); - } else if (emittedString == null) { + if (emittedString == null) { latestTimestampEmitted = -1L; latestTimestampListed = -1L; getLogger().debug("Found no recognized state keys; assuming no relevant state and resetting listing/emitted time to -1"); diff --git a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/GetHBase.java b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/GetHBase.java index 3cd81a3f5b01..6002c3cc397b 100644 --- a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/GetHBase.java +++ b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/GetHBase.java @@ -34,6 +34,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -73,7 +74,6 @@ import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; -import org.apache.nifi.util.ObjectHolder; @TriggerWhenEmpty @TriggerSerially @@ -274,8 +274,8 @@ public void onTrigger(final ProcessContext context, final ProcessSession session final Map> cellsMatchingTimestamp = new HashMap<>(); - final ObjectHolder rowsPulledHolder = new ObjectHolder<>(0L); - final ObjectHolder latestTimestampHolder = new ObjectHolder<>(minTime); + final AtomicReference rowsPulledHolder = new AtomicReference<>(0L); + final AtomicReference latestTimestampHolder = new AtomicReference<>(minTime); hBaseClientService.scan(tableName, columns, filterExpression, minTime, new ResultHandler() { diff --git a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/PutHBaseJSON.java b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/PutHBaseJSON.java index 9a57d6e4460f..4c4c2073c5c7 100644 --- a/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/PutHBaseJSON.java +++ b/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/PutHBaseJSON.java @@ -26,6 +26,7 @@ import java.util.Iterator; import java.util.List; import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; import org.apache.commons.lang3.StringUtils; import org.apache.nifi.annotation.behavior.EventDriven; @@ -46,7 +47,6 @@ import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; -import org.apache.nifi.util.ObjectHolder; import org.codehaus.jackson.JsonNode; import org.codehaus.jackson.map.ObjectMapper; @@ -166,7 +166,7 @@ protected PutFlowFile createPut(final ProcessSession session, final ProcessConte // Parse the JSON document final ObjectMapper mapper = new ObjectMapper(); - final ObjectHolder rootNodeRef = new ObjectHolder<>(null); + final AtomicReference rootNodeRef = new AtomicReference<>(null); try { session.read(flowFile, new InputStreamCallback() { @Override @@ -189,13 +189,13 @@ public void process(final InputStream in) throws IOException { } final Collection columns = new ArrayList<>(); - final ObjectHolder rowIdHolder = new ObjectHolder<>(null); + final AtomicReference rowIdHolder = new AtomicReference<>(null); // convert each field/value to a column for the put, skip over nulls and arrays final Iterator fieldNames = rootNode.getFieldNames(); while (fieldNames.hasNext()) { final String fieldName = fieldNames.next(); - final ObjectHolder fieldValueHolder = new ObjectHolder<>(null); + final AtomicReference fieldValueHolder = new AtomicReference<>(null); final JsonNode fieldNode = rootNode.get(fieldName); if (fieldNode.isNull()) { diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/SelectHiveQL.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/SelectHiveQL.java index 77ded36da4d5..9c0ebef39423 100644 --- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/SelectHiveQL.java +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/SelectHiveQL.java @@ -28,6 +28,7 @@ import java.util.List; import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.behavior.InputRequirement; @@ -47,7 +48,6 @@ import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; -import org.apache.nifi.util.LongHolder; import org.apache.nifi.util.StopWatch; import org.apache.nifi.util.hive.HiveJdbcCommon; @@ -157,7 +157,7 @@ public void onTrigger(final ProcessContext context, final ProcessSession session try (final Connection con = dbcpService.getConnection(); final Statement st = con.createStatement()) { - final LongHolder nrOfRows = new LongHolder(0L); + final AtomicLong nrOfRows = new AtomicLong(0L); if (fileToProcess == null) { fileToProcess = session.create(); } @@ -182,7 +182,7 @@ public void process(final OutputStream out) throws IOException { }); // set attribute how many rows were selected - fileToProcess = session.putAttribute(fileToProcess, RESULT_ROW_COUNT, nrOfRows.get().toString()); + fileToProcess = session.putAttribute(fileToProcess, RESULT_ROW_COUNT, String.valueOf(nrOfRows.get())); // Set MIME type on output document and add extension if (AVRO.equals(outputFormat)) { diff --git a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertAvroSchema.java b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertAvroSchema.java index ffcd653539c3..a8244d212900 100644 --- a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertAvroSchema.java +++ b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertAvroSchema.java @@ -53,7 +53,6 @@ import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.StreamCallback; import org.apache.nifi.processors.kite.AvroRecordConverter.AvroConversionException; -import org.apache.nifi.util.LongHolder; import org.kitesdk.data.DatasetException; import org.kitesdk.data.DatasetIOException; import org.kitesdk.data.SchemaNotFoundException; @@ -63,6 +62,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; +import java.util.concurrent.atomic.AtomicLong; @Tags({ "avro", "convert", "kite" }) @CapabilityDescription("Convert records from one Avro schema to another, including support for flattening and simple type conversions") @@ -291,7 +291,7 @@ public void onTrigger(ProcessContext context, final ProcessSession session) failureWriter.setCodec(CodecFactory.snappyCodec()); try { - final LongHolder written = new LongHolder(0L); + final AtomicLong written = new AtomicLong(0L); final FailureTracker failures = new FailureTracker(); final List badRecords = Lists.newLinkedList(); diff --git a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertCSVToAvro.java b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertCSVToAvro.java index 43b33ff45302..de4130ff67eb 100644 --- a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertCSVToAvro.java +++ b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertCSVToAvro.java @@ -47,7 +47,6 @@ import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.StreamCallback; import org.apache.nifi.processor.util.StandardValidators; -import org.apache.nifi.util.LongHolder; import org.kitesdk.data.DatasetException; import org.kitesdk.data.DatasetIOException; import org.kitesdk.data.DatasetRecordException; @@ -59,6 +58,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; +import java.util.concurrent.atomic.AtomicLong; @Tags({"kite", "csv", "avro"}) @InputRequirement(Requirement.INPUT_REQUIRED) @@ -224,7 +224,7 @@ public void onTrigger(ProcessContext context, final ProcessSession session) writer.setCodec(CodecFactory.snappyCodec()); try { - final LongHolder written = new LongHolder(0L); + final AtomicLong written = new AtomicLong(0L); final FailureTracker failures = new FailureTracker(); FlowFile badRecords = session.clone(incomingCSV); diff --git a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertJSONToAvro.java b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertJSONToAvro.java index af120bfa5341..6245362d0607 100644 --- a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertJSONToAvro.java +++ b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertJSONToAvro.java @@ -39,7 +39,6 @@ import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.StreamCallback; -import org.apache.nifi.util.LongHolder; import org.kitesdk.data.DatasetException; import org.kitesdk.data.DatasetIOException; import org.kitesdk.data.DatasetRecordException; @@ -50,6 +49,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; +import java.util.concurrent.atomic.AtomicLong; @Tags({"kite", "json", "avro"}) @InputRequirement(Requirement.INPUT_REQUIRED) @@ -132,7 +132,7 @@ public void onTrigger(final ProcessContext context, final ProcessSession session writer.setCodec(CodecFactory.snappyCodec()); try { - final LongHolder written = new LongHolder(0L); + final AtomicLong written = new AtomicLong(0L); final FailureTracker failures = new FailureTracker(); FlowFile badRecords = session.clone(incomingJSON); diff --git a/nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/image/ExtractImageMetadata.java b/nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/image/ExtractImageMetadata.java index 1dd9241185dc..06dd52b735e8 100644 --- a/nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/image/ExtractImageMetadata.java +++ b/nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/image/ExtractImageMetadata.java @@ -44,13 +44,13 @@ import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; -import org.apache.nifi.util.ObjectHolder; import com.drew.imaging.ImageMetadataReader; import com.drew.imaging.ImageProcessingException; import com.drew.metadata.Directory; import com.drew.metadata.Metadata; import com.drew.metadata.Tag; +import java.util.concurrent.atomic.AtomicReference; @InputRequirement(Requirement.INPUT_REQUIRED) @Tags({"Exif", "Exchangeable", "image", "file", "format", "JPG", "GIF", "PNG", "BMP", "metadata","IPTC", "XMP"}) @@ -116,7 +116,7 @@ public void onTrigger(final ProcessContext context, final ProcessSession session } final ComponentLog logger = this.getLogger(); - final ObjectHolder value = new ObjectHolder<>(null); + final AtomicReference value = new AtomicReference<>(null); final Integer max = context.getProperty(MAX_NUMBER_OF_ATTRIBUTES).asInteger(); try { diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java index c6c4b159bf83..0d7886db260c 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java @@ -113,7 +113,6 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository { - public static final String DEPRECATED_CLASS_NAME = "nifi.controller.repository.provenance.PersistentProvenanceRepository"; public static final String EVENT_CATEGORY = "Provenance Repository"; private static final String FILE_EXTENSION = ".prov"; private static final String TEMP_FILE_SUFFIX = ".prov.part"; diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java index 6e7c2f22a724..7445c5d80171 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java @@ -35,7 +35,6 @@ import org.apache.nifi.provenance.search.QuerySubmission; import org.apache.nifi.provenance.search.SearchTerm; import org.apache.nifi.provenance.search.SearchableField; -import org.apache.nifi.util.IntegerHolder; import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.util.RingBuffer; import org.apache.nifi.util.RingBuffer.Filter; @@ -611,7 +610,7 @@ public QueryRunnable(final RingBuffer ringBuffer, final F @Override public void run() { // Retrieve the most recent results and count the total number of matches - final IntegerHolder matchingCount = new IntegerHolder(0); + final AtomicInteger matchingCount = new AtomicInteger(0); final List matchingRecords = new ArrayList<>(maxRecords); ringBuffer.forEach(new ForEachEvaluator() { @Override diff --git a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteProvenanceReportingTask.java b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteProvenanceReportingTask.java index 268b7349570d..a6eb66295bb1 100644 --- a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteProvenanceReportingTask.java +++ b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteProvenanceReportingTask.java @@ -49,10 +49,8 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.TimeZone; import java.util.UUID; import java.util.concurrent.TimeUnit; diff --git a/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/PutSolrContentStream.java b/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/PutSolrContentStream.java index ca1628647f49..76a70d4a0f54 100644 --- a/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/PutSolrContentStream.java +++ b/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/PutSolrContentStream.java @@ -32,6 +32,7 @@ import java.util.SortedMap; import java.util.TreeMap; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; import org.apache.nifi.annotation.behavior.DynamicProperty; import org.apache.nifi.annotation.behavior.InputRequirement; @@ -47,7 +48,6 @@ import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; -import org.apache.nifi.util.ObjectHolder; import org.apache.nifi.util.StopWatch; import org.apache.solr.client.solrj.SolrServerException; import org.apache.solr.client.solrj.request.ContentStreamUpdateRequest; @@ -166,8 +166,8 @@ public void onTrigger(final ProcessContext context, final ProcessSession session return; } - final ObjectHolder error = new ObjectHolder<>(null); - final ObjectHolder connectionError = new ObjectHolder<>(null); + final AtomicReference error = new AtomicReference<>(null); + final AtomicReference connectionError = new AtomicReference<>(null); final boolean isSolrCloud = SOLR_TYPE_CLOUD.equals(context.getProperty(SOLR_TYPE).getValue()); final String collection = context.getProperty(COLLECTION).evaluateAttributeExpressions(flowFile).getValue(); diff --git a/nifi-nar-bundles/nifi-splunk-bundle/nifi-splunk-processors/src/main/java/org/apache/nifi/processors/splunk/PutSplunk.java b/nifi-nar-bundles/nifi-splunk-bundle/nifi-splunk-processors/src/main/java/org/apache/nifi/processors/splunk/PutSplunk.java index c461b0feb8e6..9e4aa125b4b4 100644 --- a/nifi-nar-bundles/nifi-splunk-bundle/nifi-splunk-processors/src/main/java/org/apache/nifi/processors/splunk/PutSplunk.java +++ b/nifi-nar-bundles/nifi-splunk-bundle/nifi-splunk-processors/src/main/java/org/apache/nifi/processors/splunk/PutSplunk.java @@ -39,7 +39,6 @@ import org.apache.nifi.stream.io.ByteCountingInputStream; import org.apache.nifi.stream.io.StreamUtils; import org.apache.nifi.stream.io.util.NonThreadSafeCircularBuffer; -import org.apache.nifi.util.LongHolder; import javax.net.ssl.SSLContext; import java.io.IOException; @@ -50,6 +49,7 @@ import java.util.Collection; import java.util.List; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; @InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED) @Tags({"splunk", "logs", "tcp", "udp"}) @@ -219,7 +219,7 @@ private void processDelimitedMessages(final ProcessContext context, final Proces // some pattern. We can use this to search for the delimiter as we read through the stream of bytes in the FlowFile final NonThreadSafeCircularBuffer buffer = new NonThreadSafeCircularBuffer(delimiterBytes); - final LongHolder messagesSent = new LongHolder(0L); + final AtomicLong messagesSent = new AtomicLong(0L); final FlowFileMessageBatch messageBatch = new FlowFileMessageBatch(session, flowFile); activeBatches.add(messageBatch); diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java index 2f1ff43a4fbd..21172e9a534a 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java @@ -31,7 +31,6 @@ import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processors.standard.util.JsonPathExpressionValidator; import org.apache.nifi.stream.io.BufferedInputStream; -import org.apache.nifi.util.ObjectHolder; import java.io.IOException; import java.io.InputStream; @@ -39,6 +38,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.concurrent.atomic.AtomicReference; /** * Provides common functionality used for processors interacting and manipulating JSON data via JsonPath. @@ -73,7 +73,7 @@ public abstract class AbstractJsonPathProcessor extends AbstractProcessor { static DocumentContext validateAndEstablishJsonContext(ProcessSession processSession, FlowFile flowFile) { // Parse the document once into an associated context to support multiple path evaluations if specified - final ObjectHolder contextHolder = new ObjectHolder<>(null); + final AtomicReference contextHolder = new AtomicReference<>(null); processSession.read(flowFile, new InputStreamCallback() { @Override public void process(InputStream in) throws IOException { diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/BinFiles.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/BinFiles.java deleted file mode 100644 index 3df4317c3381..000000000000 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/BinFiles.java +++ /dev/null @@ -1,354 +0,0 @@ -/* - * 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.processors.standard; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.Queue; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.TimeUnit; - -import org.apache.nifi.annotation.lifecycle.OnScheduled; -import org.apache.nifi.annotation.lifecycle.OnStopped; -import org.apache.nifi.components.PropertyDescriptor; -import org.apache.nifi.components.ValidationContext; -import org.apache.nifi.components.ValidationResult; -import org.apache.nifi.flowfile.FlowFile; -import org.apache.nifi.logging.ComponentLog; -import org.apache.nifi.processor.AbstractSessionFactoryProcessor; -import org.apache.nifi.processor.DataUnit; -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.exception.ProcessException; -import org.apache.nifi.processor.util.StandardValidators; -import org.apache.nifi.processors.standard.util.Bin; -import org.apache.nifi.processors.standard.util.BinManager; -import org.apache.nifi.processors.standard.util.FlowFileSessionWrapper; - -/** - * Base class for file-binning processors, including MergeContent. - * - * @deprecated As of release 0.5.0, replaced by - * {@link org.apache.nifi.processor.util.bin.BinFiles} - */ -@Deprecated -public abstract class BinFiles extends AbstractSessionFactoryProcessor { - - public static final PropertyDescriptor MIN_SIZE = new PropertyDescriptor.Builder() - .name("Minimum Group Size") - .description("The minimum size of for the bundle") - .required(true) - .defaultValue("0 B") - .addValidator(StandardValidators.DATA_SIZE_VALIDATOR) - .build(); - public static final PropertyDescriptor MAX_SIZE = new PropertyDescriptor.Builder() - .name("Maximum Group Size") - .description("The maximum size for the bundle. If not specified, there is no maximum.") - .required(false) - .addValidator(StandardValidators.DATA_SIZE_VALIDATOR) - .build(); - - public static final PropertyDescriptor MIN_ENTRIES = new PropertyDescriptor.Builder() - .name("Minimum Number of Entries") - .description("The minimum number of files to include in a bundle") - .required(true) - .defaultValue("1") - .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) - .build(); - public static final PropertyDescriptor MAX_ENTRIES = new PropertyDescriptor.Builder() - .name("Maximum Number of Entries") - .description("The maximum number of files to include in a bundle. If not specified, there is no maximum.") - .required(false) - .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) - .build(); - - public static final PropertyDescriptor MAX_BIN_COUNT = new PropertyDescriptor.Builder() - .name("Maximum number of Bins") - .description("Specifies the maximum number of bins that can be held in memory at any one time") - .defaultValue("100") - .required(true) - .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) - .build(); - - public static final PropertyDescriptor MAX_BIN_AGE = new PropertyDescriptor.Builder() - .name("Max Bin Age") - .description("The maximum age of a Bin that will trigger a Bin to be complete. Expected format is