You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ma...@apache.org on 2016/07/14 13:32:53 UTC

[1/6] nifi git commit: NIFI-1157 resolved deprecated nifi-api items and ripple effects

Repository: nifi
Updated Branches:
  refs/heads/master 6e5e4cf52 -> f987b2160


http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/util/ReflectionUtils.java
----------------------------------------------------------------------
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 94fc1f3..1d3de36 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 @@ package org.apache.nifi.documentation.util;
 import java.lang.annotation.Annotation;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
-import java.util.ArrayList;
-import java.util.List;
 
 import org.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 <code>instance</code> uses 1 or more parameters, those parameters must be
-     * specified by the <code>args</code> parameter. However, if more arguments are supplied by the <code>args</code> 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
+     * <code>instance</code> uses 1 or more parameters, those parameters must be
+     * specified by the <code>args</code> parameter. However, if more arguments
+     * are supplied by the <code>args</code> 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 <code>true</code> if all appropriate methods were invoked and returned without throwing an Exception, <code>false</code> if one of the methods threw an Exception or could not be
-     * invoked; if <code>false</code> is returned, an error will have been logged.
+     * @return <code>true</code> if all appropriate methods were invoked and
+     * returned without throwing an Exception, <code>false</code> if one of the
+     * methods threw an Exception or could not be invoked; if <code>false</code>
+     * is returned, an error will have been logged.
      */
-    public static boolean quietlyInvokeMethodsWithAnnotations(
-            final Class<? extends Annotation> preferredAnnotation, final Class<? extends Annotation> alternateAnnotation, final Object instance, final ComponentLog logger, final Object... args) {
-        final List<Class<? extends Annotation>> annotationClasses = new ArrayList<>(alternateAnnotation == null ? 1 : 2);
-        annotationClasses.add(preferredAnnotation);
-        if (alternateAnnotation != null) {
-            annotationClasses.add(alternateAnnotation);
-        }
+    public static boolean quietlyInvokeMethodsWithAnnotation(
+            final Class<? extends Annotation> annotation, final Object instance, final ComponentLog logger, final Object... args) {
 
-        boolean annotationFound = false;
-        for (final Class<? extends Annotation> 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;
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FileSystemSwapManager.java
----------------------------------------------------------------------
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 5234f0e..5a33eec 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.nio.file.Path;
 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 class FileSystemSwapManager implements FlowFileSwapManager {
             for (final FlowFileRecord flowFile : toSwap) {
                 out.writeLong(flowFile.getId());
                 out.writeLong(flowFile.getEntryDate());
-
-                final Set<String> 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 @@ public class FileSystemSwapManager implements FlowFileSwapManager {
 
                 if (serializationVersion > 1) {
                     // Lineage information was added in version 2
-                    final int numLineageIdentifiers = in.readInt();
-                    final Set<String> 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();

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
----------------------------------------------------------------------
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 0c90c50..76b946f 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 class FlowController implements EventAccess, ControllerServiceProvider, R
 
         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 class FlowController implements EventAccess, ControllerServiceProvider, R
 
         final String parentUUID = event.getFlowFileUuid();
 
-        // Create the FlowFile Record
-        final Set<String> 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 class FlowController implements EventAccess, ControllerServiceProvider, R
             .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

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
----------------------------------------------------------------------
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 3ea2b6b..b232f26 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 class StandardProcessorNode extends ProcessorNode implements Connectable
         identifier = new AtomicReference<>(uuid);
         destinations = new HashMap<>();
         connections = new HashMap<>();
-        incomingConnectionsRef = new AtomicReference<List<Connection>>(new ArrayList<Connection>());
+        incomingConnectionsRef = new AtomicReference<>(new ArrayList<>());
         lossTolerant = new AtomicBoolean(false);
         final Set<Relationship> emptySetOfRelationships = new HashSet<>();
         undefinedRelationshipsToTerminate = new AtomicReference<>(emptySetOfRelationships);
@@ -169,20 +169,12 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
         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 class StandardProcessorNode extends ProcessorNode implements Connectable
      * @return the value of the processor's {@link CapabilityDescription}
      *         annotation, if one exists, else <code>null</code>.
      */
-    @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 class StandardProcessorNode extends ProcessorNode implements Connectable
                             @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;
                                 }
                             }

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
----------------------------------------------------------------------
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 9117a16..bcb3feb 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.ProcessScheduler;
 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 abstract class AbstractReportingTaskNode extends AbstractConfiguredCompon
     @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() {

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardFlowFileRecord.java
----------------------------------------------------------------------
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 cd1ba14..607ccfd 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 @@ import org.apache.nifi.flowfile.attributes.CoreAttributes;
  *
  */
 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<String> lineageIdentifiers;
     private final long size;
     private final long penaltyExpirationMs;
     private final Map<String, String> attributes;
@@ -64,7 +61,6 @@ public final class StandardFlowFileRecord implements FlowFile, FlowFileRecord {
         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;
@@ -84,11 +80,6 @@ public final class StandardFlowFileRecord implements FlowFile, FlowFileRecord {
     }
 
     @Override
-    public Set<String> getLineageIdentifiers() {
-        return Collections.unmodifiableSet(lineageIdentifiers);
-    }
-
-    @Override
     public long getLineageStartDate() {
         return lineageStartDate;
     }
@@ -196,25 +187,6 @@ public final class StandardFlowFileRecord implements FlowFile, FlowFileRecord {
             return this;
         }
 
-        public Builder lineageIdentifiers(final Collection<String> 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 final class StandardFlowFileRecord implements FlowFile, FlowFileRecord {
             bLineageStartDate = specFlowFile.getLineageStartDate();
             bLineageStartIndex = specFlowFile.getLineageStartIndex();
             bLineageIdentifiers.clear();
-            bLineageIdentifiers.addAll(specFlowFile.getLineageIdentifiers());
             bPenaltyExpirationMs = specFlowFile.getPenaltyExpirationMillis();
             bSize = specFlowFile.getSize();
             bAttributes.putAll(specFlowFile.getAttributes());

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
----------------------------------------------------------------------
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 b5da072..1e2d634 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 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
             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 final class StandardProcessSession implements ProcessSession, ProvenanceE
             newAttributes.put(key, value);
         }
 
-        final Set<String> 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 final class StandardProcessSession implements ProcessSession, ProvenanceE
         long lineageStartDate = 0L;
         final Set<String> 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 final class StandardProcessSession implements ProcessSession, ProvenanceE
 
         final FlowFileRecord fFile = new StandardFlowFileRecord.Builder().id(context.getNextFlowFileSequence())
             .addAttributes(newAttributes)
-            .lineageIdentifiers(lineageIdentifiers)
             .lineageStart(lineageStartDate, lineageStartIndex)
             .build();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java
----------------------------------------------------------------------
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 0f40cbb..dae1cff 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 class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
     }
 
     private static class WriteAheadRecordSerde implements SerDe<RepositoryRecord> {
-        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 class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
 
             out.writeLong(getRecordIdentifier(record));
             out.writeLong(flowFile.getEntryDate());
-
-            final Set<String> 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 class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
 
             if (version > 1) {
                 // read the lineage identifiers and lineage start date, which were added in version 2.
-                final int numLineageIds = in.readInt();
-                final Set<String> 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 class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
 
             if (version > 1) {
                 // read the lineage identifiers and lineage start date, which were added in version 2.
-                final int numLineageIds = in.readInt();
-                final Set<String> 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;

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java
----------------------------------------------------------------------
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 c47d6aa..d3410cc 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 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent {
             } 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);
                     }
                 }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
----------------------------------------------------------------------
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 7856dcd..0c1979c 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.ConfigurationContext;
 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 class StandardControllerServiceNode extends AbstractConfiguredComponent i
     @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

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java
----------------------------------------------------------------------
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 04e3f60..8bc7d99 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 class ContinuallyRunConnectableTask implements Callable<Boolean> {
             } 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);
                     }
                 }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java
----------------------------------------------------------------------
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 77f60b5..1dc08c2 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 class ReportingTaskWrapper implements Runnable {
                 // 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 {

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
----------------------------------------------------------------------
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 a931841..c8d9d25 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 @@ public final class StandardProcessGroup implements ProcessGroup {
         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 final class StandardProcessGroup implements ProcessGroup {
 
             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);
             }

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFileSystemSwapManager.java
----------------------------------------------------------------------
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 c27fad3..50bc874 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.Collections;
 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 class TestFileSystemSwapManager {
             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());
         }
@@ -199,11 +197,6 @@ public class TestFileSystemSwapManager {
         }
 
         @Override
-        public Set<String> getLineageIdentifiers() {
-            return Collections.emptySet();
-        }
-
-        @Override
         public boolean isPenalized() {
             return false;
         }

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardFlowFileQueue.java
----------------------------------------------------------------------
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 85d9838..6e368de 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 class TestStandardFlowFileQueue {
         public String swapOut(List<FlowFileRecord> flowFiles, FlowFileQueue flowFileQueue) throws IOException {
             swapOutCalledCount++;
             final String location = UUID.randomUUID().toString();
-            swappedOut.put(location, new ArrayList<FlowFileRecord>(flowFiles));
+            swappedOut.put(location, new ArrayList<>(flowFiles));
             return location;
         }
 
@@ -560,7 +560,7 @@ public class TestStandardFlowFileQueue {
 
         @Override
         public List<String> recoverSwapLocations(FlowFileQueue flowFileQueue) throws IOException {
-            return new ArrayList<String>(swappedOut.keySet());
+            return new ArrayList<>(swappedOut.keySet());
         }
 
         @Override
@@ -610,7 +610,7 @@ public class TestStandardFlowFileQueue {
         }
 
         public TestFlowFile(final long size) {
-            this(new HashMap<String, String>(), size);
+            this(new HashMap<>(), size);
         }
 
         public TestFlowFile(final Map<String, String> attributes, final long size) {
@@ -648,11 +648,6 @@ public class TestStandardFlowFileQueue {
         }
 
         @Override
-        public Set<String> getLineageIdentifiers() {
-            return Collections.emptySet();
-        }
-
-        @Override
         public boolean isPenalized() {
             return false;
         }

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java
----------------------------------------------------------------------
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 3b827eb..c6c4b15 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");

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/StandardRecordReader.java
----------------------------------------------------------------------
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 a9d7bf7..2db9ed3 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.io.InputStream;
 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 class StandardRecordReader implements RecordReader {
         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 @@ public class StandardRecordReader implements RecordReader {
         final Map<String, String> attrs = readAttributes(dis, false);
 
         builder.setFlowFileEntryDate(System.currentTimeMillis());
-        builder.setLineageIdentifiers(Collections.<String>emptySet());
         builder.setLineageStartDate(-1L);
         builder.setAttributes(Collections.<String, String>emptyMap(), attrs);
         builder.setCurrentContentClaim(null, null, null, null, fileSize);
@@ -288,10 +285,11 @@ public class StandardRecordReader implements RecordReader {
         final Long flowFileEntryDate = dis.readLong();
         builder.setEventDuration(dis.readLong());
 
-        final Set<String> 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 class StandardRecordReader implements RecordReader {
         }
 
         builder.setFlowFileEntryDate(flowFileEntryDate);
-        builder.setLineageIdentifiers(lineageIdentifiers);
         builder.setLineageStartDate(lineageStartDate);
         builder.setStorageLocation(filename, startOffset);
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/StandardRecordWriter.java
----------------------------------------------------------------------
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 a5c121a..35832c4 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 class StandardRecordWriter implements RecordWriter {
             out.writeLong(record.getEventTime());
             out.writeLong(record.getFlowFileEntryDate());
             out.writeLong(record.getEventDuration());
-
-            writeUUIDs(out, record.getLineageIdentifiers());
             out.writeLong(record.getLineageStartDate());
 
             writeNullableString(out, record.getComponentId());

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexingAction.java
----------------------------------------------------------------------
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 46be391..ba99058 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 class IndexingAction {
                 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()) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LineageQuery.java
----------------------------------------------------------------------
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 3a2d6e1..2706082 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<ProvenanceEventRecord> computeLineageForFlowFiles(final PersistentProvenanceRepository repo, final IndexManager indexManager, final File indexDirectory,
-        final String lineageIdentifier, final Collection<String> flowFileUuids, final int maxAttributeChars) throws IOException {
+            final String lineageIdentifier, final Collection<String> 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 class LineageQuery {
                     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 class LineageQuery {
 
                 final DocsReader docsReader = new DocsReader();
                 final Set<ProvenanceEventRecord> 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 class LineageQuery {
         } 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);
             }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestUtil.java
----------------------------------------------------------------------
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 a34d78b..514a43e 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;
 
@@ -39,11 +37,6 @@ public class TestUtil {
             }
 
             @Override
-            public Set<String> getLineageIdentifiers() {
-                return new HashSet<String>();
-            }
-
-            @Override
             public long getLineageStartDate() {
                 return System.currentTimeMillis();
             }

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java
----------------------------------------------------------------------
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 ce53393..6e7c2f2 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.Date;
 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;
@@ -710,11 +709,6 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
         }
 
         @Override
-        public Set<String> getLineageIdentifiers() {
-            return record.getLineageIdentifiers();
-        }
-
-        @Override
         public long getFileSize() {
             return record.getFileSize();
         }

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/test/java/org/apache/nifi/provenance/TestVolatileProvenanceRepository.java
----------------------------------------------------------------------
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 ccc8892..7db650d 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 org.junit.Test;
 
 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;
@@ -134,11 +132,6 @@ public class TestVolatileProvenanceRepository {
             }
 
             @Override
-            public Set<String> getLineageIdentifiers() {
-                return new HashSet<String>();
-            }
-
-            @Override
             public long getLineageStartDate() {
                 return System.currentTimeMillis();
             }

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteProvenanceReportingTask.java
----------------------------------------------------------------------
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 4a897f7..268b734 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 @@ public class SiteToSiteProvenanceReportingTask extends AbstractSiteToSiteReporti
         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<String> 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());

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/test/java/org/apache/nifi/reporting/TestSiteToSiteProvenanceReportingTask.java
----------------------------------------------------------------------
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 265bdd0..a048f5b 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.io.IOException;
 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 class TestSiteToSiteProvenanceReportingTask {
         final Map<String, String> prevAttrs = new HashMap<>();
         attributes.put("filename", "1234.xyz");
 
-        final Set<String> 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 class TestSiteToSiteProvenanceReportingTask {
         builder.setAttributes(prevAttrs, attributes);
         builder.setComponentId("1234");
         builder.setComponentType("dummy processor");
-        builder.setLineageIdentifiers(lineageIdentifiers);
         final ProvenanceEventRecord event = builder.build();
 
         final List<byte[]> dataSent = new ArrayList<>();


[6/6] nifi git commit: NIFI-1157 searched for and resolved all remaining references to deprecated items that were clearly addressable.

Posted by ma...@apache.org.
NIFI-1157 searched for and resolved all remaining references to deprecated items that were clearly addressable.


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/f987b216
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/f987b216
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/f987b216

Branch: refs/heads/master
Commit: f987b216090f29719976ed1693be2ea358523aa5
Parents: 961be21
Author: joewitt <jo...@apache.org>
Authored: Thu Jul 14 00:51:04 2016 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Thu Jul 14 09:32:35 2016 -0400

----------------------------------------------------------------------
 .../provenance/PlaceholderProvenanceEvent.java  |   1 -
 .../StandardProvenanceEventRecord.java          |   1 -
 .../nifi/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 ----
 .../java/org/apache/nifi/util/ObjectHolder.java |  44 ---
 .../java/org/apache/nifi/util/MockFlowFile.java |   1 -
 .../nifi/util/StandardProcessorTestRunner.java  |  17 -
 .../java/org/apache/nifi/util/TestRunner.java   |   7 -
 .../util/TestStandardProcessorTestRunner.java   |  20 --
 .../apache/nifi/processors/avro/SplitAvro.java  |   6 +-
 .../processors/cassandra/QueryCassandra.java    |   6 +-
 .../nifi/controller/FileSystemSwapManager.java  |   2 -
 .../apache/nifi/controller/FlowController.java  |   2 -
 .../nifi/controller/StandardFlowFileQueue.java  |   1 -
 .../nifi/controller/StandardProcessorNode.java  |   1 -
 .../repository/FileSystemRepository.java        |   3 +-
 .../repository/StandardProcessSession.java      |  50 ++-
 .../repository/io/ByteCountingInputStream.java  |  11 +-
 .../repository/io/ByteCountingOutputStream.java |  11 +-
 .../controller/repository/io/LongHolder.java    |  46 ---
 .../scheduling/EventDrivenSchedulingAgent.java  |   1 -
 .../scheduling/StandardProcessScheduler.java    |   2 -
 .../StandardControllerServiceProvider.java      |   4 +-
 .../tasks/ContinuallyRunConnectableTask.java    |   1 -
 .../controller/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 ----
 .../org/apache/nifi/web/api/dto/DtoFactory.java |  22 +-
 .../nifi/web/OptimisticLockingManager.java      |  57 ---
 .../apache/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 +-
 .../nifi/processors/kite/ConvertAvroSchema.java |   4 +-
 .../nifi/processors/kite/ConvertCSVToAvro.java  |   4 +-
 .../nifi/processors/kite/ConvertJSONToAvro.java |   4 +-
 .../processors/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 +-
 .../nifi/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 +-
 .../nifi/processors/standard/MergeContent.java  |   8 +-
 .../nifi/processors/standard/PostHTTP.java      |   3 +-
 .../processors/standard/PutFileTransfer.java    |   4 +-
 .../nifi/processors/standard/PutSyslog.java     |   4 +-
 .../processors/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 +-
 .../nifi/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 +-
 .../util/crypto/OpenSSLPKCS5CipherProvider.java |  16 -
 .../standard/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 --
 .../java/org/apache/nifi/ssl/TestProcessor.java |   4 +-
 87 files changed, 129 insertions(+), 1844 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/PlaceholderProvenanceEvent.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 083b13e..26696c8 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 @@ package org.apache.nifi.provenance;
 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

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java
----------------------------------------------------------------------
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 7b0c91f..34de366 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 @@ public final class StandardProvenanceEventRecord implements ProvenanceEventRecor
         updatedAttributes = builder.updatedAttributes == null ? Collections.<String, String>emptyMap() : Collections.unmodifiableMap(builder.updatedAttributes);
 
         sourceQueueIdentifier = builder.sourceQueueIdentifier;
-    
     }
 
     public String getStorageFilename() {

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java
----------------------------------------------------------------------
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 2419c89..47d5d50 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.URI;
 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 class StandardValidators {
         }
     }
 
-    /**
-     * 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<? extends ControllerService> 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<ValidationResult> 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();
-            }
-        };
-    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/BooleanHolder.java
----------------------------------------------------------------------
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 8283389..0000000
--- 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 <code>final</code> 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<Boolean> {
-
-    public BooleanHolder(final boolean initialValue) {
-        super(initialValue);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/IntegerHolder.java
----------------------------------------------------------------------
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 8abfdb1..0000000
--- 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 <code>final</code> 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<Integer> {
-
-    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);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/LongHolder.java
----------------------------------------------------------------------
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 723463a..0000000
--- 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 <code>final</code> 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<Long> {
-
-    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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/ObjectHolder.java
----------------------------------------------------------------------
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 c577d6f..0000000
--- 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<T> {
-
-    private T value;
-
-    public ObjectHolder(final T initialValue) {
-        this.value = initialValue;
-    }
-
-    public T get() {
-        return value;
-    }
-
-    public void set(T value) {
-        this.value = value;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-mock/src/main/java/org/apache/nifi/util/MockFlowFile.java
----------------------------------------------------------------------
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 02806b0..516c8a4 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.nio.file.StandardOpenOption;
 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;

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
----------------------------------------------------------------------
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 7d49f0f..b80c09f 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.ProcessSessionFactory;
 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 class StandardProcessorTestRunner implements TestRunner {
         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 class StandardProcessorTestRunner implements TestRunner {
 
     @Override
     public void addControllerService(final String identifier, final ControllerService service, final Map<String, String> 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);

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java
----------------------------------------------------------------------
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 c3bedb1..44d7f9f 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.ProcessSessionFactory;
 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;
 
@@ -486,12 +485,6 @@ public interface TestRunner {
     List<MockFlowFile> 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
      */
     QueueSize getQueueSize();

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-mock/src/test/java/org/apache/nifi/util/TestStandardProcessorTestRunner.java
----------------------------------------------------------------------
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 0e41e89..f5b28aa 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 class TestStandardProcessorTestRunner {
         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;

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/SplitAvro.java
----------------------------------------------------------------------
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 38e3a0d..ac6936f 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.HashSet;
 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.InputStreamCallback;
 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 class SplitAvro extends AbstractProcessor {
         @Override
         public List<FlowFile> split(final ProcessSession session, final FlowFile originalFlowFile, final SplitWriter splitWriter) {
             final List<FlowFile> childFlowFiles = new ArrayList<>();
-            final ObjectHolder<GenericRecord> recordHolder = new ObjectHolder<>(null);
+            final AtomicReference<GenericRecord> recordHolder = new AtomicReference<>(null);
 
             session.read(originalFlowFile, new InputStreamCallback() {
                 @Override
@@ -239,7 +239,7 @@ public class SplitAvro extends AbstractProcessor {
                     try (final InputStream in = new BufferedInputStream(rawIn);
                          final DataFileStream<GenericRecord> reader = new DataFileStream<>(in, new GenericDatumReader<GenericRecord>())) {
 
-                        final ObjectHolder<String> codec = new ObjectHolder<>(reader.getMetaString(DataFileConstants.CODEC));
+                        final AtomicReference<String> codec = new AtomicReference<>(reader.getMetaString(DataFileConstants.CODEC));
                         if (codec.get() == null) {
                             codec.set(DataFileConstants.NULL_CODEC);
                         }

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/QueryCassandra.java
----------------------------------------------------------------------
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 3182958..ca7f690 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.Relationship;
 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.TimeZone;
 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 class QueryCassandra extends AbstractCassandraProcessor {
             // 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 class QueryCassandra extends AbstractCassandraProcessor {
             });
 
             // 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()});

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FileSystemSwapManager.java
----------------------------------------------------------------------
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 5a33eec..a4c267c 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 class FileSystemSwapManager implements FlowFileSwapManager {
         }
     }
 
-
-    @SuppressWarnings("deprecation")
     public static int serializeFlowFiles(final List<FlowFileRecord> toSwap, final FlowFileQueue queue, final String swapLocation, final OutputStream destination) throws IOException {
         if (toSwap == null || toSwap.isEmpty()) {
             return 0;

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
----------------------------------------------------------------------
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 76b946f..80b89dd 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 class FlowController implements EventAccess, ControllerServiceProvider, R
      * @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 class FlowController implements EventAccess, ControllerServiceProvider, R
         return replayFlowFile(record, user);
     }
 
-    @SuppressWarnings("deprecation")
     public ProvenanceEventRecord replayFlowFile(final ProvenanceEventRecord event, final NiFiUser user) throws IOException {
         if (event == null) {
             throw new NullPointerException();

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java
----------------------------------------------------------------------
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 aa77765..f391da5 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 @@ public final class StandardFlowFileQueue implements FlowFileQueue {
         }
 
         @Override
-        @SuppressWarnings("deprecation")
         public int compare(final FlowFileRecord f1, final FlowFileRecord f2) {
             int returnVal = 0;
             final boolean f1Penalized = f1.isPenalized();

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
----------------------------------------------------------------------
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 b232f26..9d2c45c 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 class StandardProcessorNode extends ProcessorNode implements Connectable
             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,

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java
----------------------------------------------------------------------
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 210fcca..673440f 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.ByteCountingOutputStream;
 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 @@ public class FileSystemRepository implements ContentRepository {
             final Callable<Long> scanContainer = new Callable<Long>() {
                 @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.

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
----------------------------------------------------------------------
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 1e2d634..0a2f8c9 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.DisableOnCloseOutputStream;
 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 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
 
         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 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
         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 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
                 }
 
                 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 final class StandardProcessSession implements ProcessSession, ProvenanceE
                     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 final class StandardProcessSession implements ProcessSession, ProvenanceE
         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 final class StandardProcessSession implements ProcessSession, ProvenanceE
             destroyContent(newClaim);
             throw t;
         } finally {
-            bytesWritten.increment(writtenHolder.getValue());
+            bytesWritten.getAndAdd(writtenHolder.get());
         }
 
         removeTemporaryClaim(record);
@@ -2150,7 +2149,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
             .fromFlowFile(record.getCurrent())
             .contentClaim(newClaim)
             .contentClaimOffset(0)
-            .size(writtenHolder.getValue())
+            .size(writtenHolder.get())
             .build();
 
         record.setWorking(newFile);
@@ -2178,7 +2177,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
 
                     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 final class StandardProcessSession implements ProcessSession, ProvenanceE
         } finally {
             if (outStream != null) {
                 final long bytesWrittenThisIteration = outStream.getBytesWritten() - originalByteWrittenCount;
-                bytesWritten.increment(bytesWrittenThisIteration);
+                bytesWritten.getAndAdd(bytesWrittenThisIteration);
             }
         }
 
@@ -2313,7 +2312,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
         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 final class StandardProcessSession implements ProcessSession, ProvenanceE
             destroyContent(newClaim);
             throw t;
         } finally {
-            bytesWritten.increment(writtenHolder.getValue());
+            bytesWritten.getAndAdd(writtenHolder.get());
         }
 
         removeTemporaryClaim(record);
@@ -2373,7 +2372,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
             .fromFlowFile(record.getCurrent())
             .contentClaim(newClaim)
             .contentClaimOffset(0L)
-            .size(writtenHolder.getValue())
+            .size(writtenHolder.get())
             .build();
 
         record.setWorking(newFile);
@@ -2405,8 +2404,8 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
         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 final class StandardProcessSession implements ProcessSession, ProvenanceE
                 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 final class StandardProcessSession implements ProcessSession, ProvenanceE
             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 final class StandardProcessSession implements ProcessSession, ProvenanceE
         // 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<String> lineageIdentifiers = new HashSet<>();
         for (final FlowFile parent : parents) {
 
             final long parentLineageStartDate = parent.getLineageStartDate();
@@ -2785,8 +2783,8 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
 
             this.removedCount += session.removedCount;
             this.removedBytes += session.removedBytes;
-            this.bytesRead += session.bytesRead.getValue();
-            this.bytesWritten += session.bytesWritten.getValue();
+            this.bytesRead += session.bytesRead.get();
+            this.bytesWritten += session.bytesWritten.get();
             this.flowFilesIn += session.flowFilesIn;
             this.flowFilesOut += session.flowFilesOut;
             this.contentSizeIn += session.contentSizeIn;

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/ByteCountingInputStream.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/ByteCountingInputStream.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/ByteCountingInputStream.java
index 6cd5d4a..7de25ac 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/ByteCountingInputStream.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/ByteCountingInputStream.java
@@ -18,14 +18,15 @@ package org.apache.nifi.controller.repository.io;
 
 import java.io.IOException;
 import java.io.InputStream;
+import java.util.concurrent.atomic.AtomicLong;
 
 public class ByteCountingInputStream extends InputStream {
 
-    private final LongHolder bytesReadHolder;
+    private final AtomicLong bytesReadHolder;
     private final InputStream in;
     private long bytesSkipped = 0L;
 
-    public ByteCountingInputStream(final InputStream in, final LongHolder longHolder) {
+    public ByteCountingInputStream(final InputStream in, final AtomicLong longHolder) {
         this.in = in;
         this.bytesReadHolder = longHolder;
     }
@@ -34,7 +35,7 @@ public class ByteCountingInputStream extends InputStream {
     public int read() throws IOException {
         final int fromSuper = in.read();
         if (fromSuper >= 0) {
-            bytesReadHolder.increment(1);
+            bytesReadHolder.getAndIncrement();
         }
         return fromSuper;
     }
@@ -43,7 +44,7 @@ public class ByteCountingInputStream extends InputStream {
     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 class ByteCountingInputStream extends InputStream {
     }
 
     public long getBytesRead() {
-        return bytesReadHolder.getValue();
+        return bytesReadHolder.get();
     }
 
     public long getBytesSkipped() {

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/ByteCountingOutputStream.java
----------------------------------------------------------------------
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 4e727e9..7c778a2 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 @@ package org.apache.nifi.controller.repository.io;
 
 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 class ByteCountingOutputStream extends OutputStream {
     @Override
     public void write(int b) throws IOException {
         out.write(b);
-        bytesWrittenHolder.increment(1);
+        bytesWrittenHolder.getAndIncrement();
     }
 
     @Override
@@ -43,11 +44,11 @@ public class ByteCountingOutputStream extends OutputStream {
     @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

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/LongHolder.java
----------------------------------------------------------------------
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 bd5fec1..0000000
--- 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;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java
----------------------------------------------------------------------
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 d3410cc..262ac77 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 class EventDrivenSchedulingAgent extends AbstractSchedulingAgent {
             }
         }
 
-        @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) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
----------------------------------------------------------------------
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 1729d4a..53fc726 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 final class StandardProcessScheduler implements ProcessScheduler {
         scheduleState.setScheduled(true);
 
         final Runnable startReportingTaskRunnable = new Runnable() {
-            @SuppressWarnings("deprecation")
             @Override
             public void run() {
                 final long lastStopTime = scheduleState.getLastStopTime();
@@ -243,7 +242,6 @@ public final class StandardProcessScheduler implements ProcessScheduler {
         taskNode.setScheduledState(ScheduledState.STOPPED);
 
         final Runnable unscheduleReportingTaskRunnable = new Runnable() {
-            @SuppressWarnings("deprecation")
             @Override
             public void run() {
                 final ConfigurationContext configurationContext = taskNode.getConfigurationContext();

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
----------------------------------------------------------------------
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 22ee5cf..2c8d258 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.Iterator;
 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.SimpleProcessLogger;
 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 class StandardControllerServiceProvider implements ControllerServiceProvi
             final Class<? extends ControllerService> controllerServiceClass = rawClass.asSubclass(ControllerService.class);
 
             final ControllerService originalService = controllerServiceClass.newInstance();
-            final ObjectHolder<ControllerServiceNode> serviceNodeHolder = new ObjectHolder<>(null);
+            final AtomicReference<ControllerServiceNode> serviceNodeHolder = new AtomicReference<>(null);
             final InvocationHandler invocationHandler = new InvocationHandler() {
                 @Override
                 public Object invoke(final Object proxy, final Method method, final Object[] args) throws Throwable {

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java
----------------------------------------------------------------------
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 8bc7d99..8f1623e 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 class ContinuallyRunConnectableTask implements Callable<Boolean> {
     }
 
     @Override
-    @SuppressWarnings("deprecation")
     public Boolean call() {
         if (!scheduleState.isScheduled()) {
             return false;

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java
----------------------------------------------------------------------
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 1dc08c2..eb087a4 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 class ReportingTaskWrapper implements Runnable {
         this.scheduleState = scheduleState;
     }
 
-    @SuppressWarnings("deprecation")
     @Override
     public synchronized void run() {
         scheduleState.incrementActiveThreadCount();

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
----------------------------------------------------------------------
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 c8d9d25..8017abd 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 @@ public final class StandardProcessGroup implements ProcessGroup {
         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 final class StandardProcessGroup implements ProcessGroup {
     }
 
     @Override
-    @SuppressWarnings("deprecation")
     public void removeProcessor(final ProcessorNode processor) {
         final String id = requireNonNull(processor).getIdentifier();
         writeLock.lock();

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFileSystemSwapManager.java
----------------------------------------------------------------------
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 50bc874..4a590f2 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 class TestFileSystemSwapManager {
     }
 
     @Test
-    @SuppressWarnings("deprecation")
     public void testRoundTripSerializeDeserialize() throws IOException {
         final List<FlowFileRecord> toSwap = new ArrayList<>(10000);
         final Map<String, String> attrs = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardFlowFileQueue.java
----------------------------------------------------------------------
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 6e368de..32c1dc6 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 class TestStandardFlowFileQueue {
         }
 
         @Override
-        @SuppressWarnings("deprecation")
         public SwapSummary getSwapSummary(String swapLocation) throws IOException {
             final List<FlowFileRecord> flowFiles = swappedOut.get(swapLocation);
             if (flowFiles == null) {
@@ -668,7 +667,6 @@ public class TestStandardFlowFileQueue {
         }
 
         @Override
-        @SuppressWarnings("deprecation")
         public int compareTo(final FlowFile o) {
             return Long.compare(id, o.getId());
         }

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java
----------------------------------------------------------------------
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 4094ca4..55b7426 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 class TestWriteAheadFlowFileRepository {
     }
 
     @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 class TestWriteAheadFlowFileRepository {
         }
 
         @Override
-        @SuppressWarnings("deprecation")
         public SwapSummary getSwapSummary(String swapLocation) throws IOException {
             List<FlowFileRecord> records = null;
             for (final Map<String, List<FlowFileRecord>> swapMap : swappedRecords.values()) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/util/FileUtils.java
----------------------------------------------------------------------
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 c02b83f..a9943b8 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
@@ -103,77 +103,6 @@ public class FileUtils {
      * @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
-     * 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
      * @throws IOException if abstract pathname does not denote a directory,
      * or if an I/O error occurs
      */

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
----------------------------------------------------------------------
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 7396a34..e22ba4b 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 @@ public final class DtoFactory {
     /**
      * 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<String> getTags(final Class<?> cls) {
         final Set<String> tags = new HashSet<>();
         final Tags tagsAnnotation = cls.getAnnotation(Tags.class);
@@ -1951,13 +1942,6 @@ public final class DtoFactory {
             for (final String tag : tagsAnnotation.value()) {
                 tags.add(tag);
             }
-        } else {
-            final org.apache.nifi.processor.annotation.Tags deprecatedTagsAnnotation = cls.getAnnotation(org.apache.nifi.processor.annotation.Tags.class);
-            if (deprecatedTagsAnnotation != null) {
-                for (final String tag : deprecatedTagsAnnotation.value()) {
-                    tags.add(tag);
-                }
-            }
         }
 
         return tags;
@@ -2129,7 +2113,6 @@ public final class DtoFactory {
      * @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 final class DtoFactory {
         dto.setFlowFileUuid(node.getFlowFileUuid());
         dto.setParentUuids(node.getParentUuids());
         dto.setChildUuids(node.getChildUuids());
-        dto.setClusterNodeIdentifier(node.getClusterNodeIdentifier());
         return dto;
     }
 
@@ -2150,7 +2132,6 @@ public final class DtoFactory {
      * @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 final class DtoFactory {
         dto.setTimestamp(new Date(node.getTimestamp()));
         dto.setMillis(node.getTimestamp());
         dto.setFlowFileUuid(node.getFlowFileUuid());
-        dto.setClusterNodeIdentifier(node.getClusterNodeIdentifier());
         return dto;
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/OptimisticLockingManager.java
----------------------------------------------------------------------
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 b205590..0000000
--- 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 <T> type of snapshot
-     * @param revision revision
-     * @param configurationRequest request
-     * @return snapshot
-     */
-    <T> ConfigurationSnapshot<T> configureFlow(Revision revision, ConfigurationRequest<T> 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();
-
-}


[5/6] nifi git commit: NIFI-1157 searched for and resolved all remaining references to deprecated items that were clearly addressable.

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/ListHDFS.java
----------------------------------------------------------------------
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 f5daef2..4cb8d25 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.behavior.WritesAttributes;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
-import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.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.Relationship;
 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";
@@ -176,80 +171,6 @@ public class ListHDFS extends AbstractHadoopProcessor {
     }
 
     /**
-     * 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.
      *
      * @param statuses the eligible FileStatus objects that we could potentially list
@@ -339,13 +260,7 @@ public class ListHDFS extends AbstractHadoopProcessor {
             } 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<String, String> 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");

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/GetHBase.java
----------------------------------------------------------------------
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 3cd81a3..6002c3c 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.HashSet;
 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.Relationship;
 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 class GetHBase extends AbstractProcessor {
 
             final Map<String, Set<String>> cellsMatchingTimestamp = new HashMap<>();
 
-            final ObjectHolder<Long> rowsPulledHolder = new ObjectHolder<>(0L);
-            final ObjectHolder<Long> latestTimestampHolder = new ObjectHolder<>(minTime);
+            final AtomicReference<Long> rowsPulledHolder = new AtomicReference<>(0L);
+            final AtomicReference<Long> latestTimestampHolder = new AtomicReference<>(minTime);
 
 
             hBaseClientService.scan(tableName, columns, filterExpression, minTime, new ResultHandler() {

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-hbase-bundle/nifi-hbase-processors/src/main/java/org/apache/nifi/hbase/PutHBaseJSON.java
----------------------------------------------------------------------
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 9a57d6e..4c4c207 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.HashSet;
 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.Relationship;
 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 @@ public class PutHBaseJSON extends AbstractPutHBase {
 
         // Parse the JSON document
         final ObjectMapper mapper = new ObjectMapper();
-        final ObjectHolder<JsonNode> rootNodeRef = new ObjectHolder<>(null);
+        final AtomicReference<JsonNode> rootNodeRef = new AtomicReference<>(null);
         try {
             session.read(flowFile, new InputStreamCallback() {
                 @Override
@@ -189,13 +189,13 @@ public class PutHBaseJSON extends AbstractPutHBase {
         }
 
         final Collection<PutColumn> columns = new ArrayList<>();
-        final ObjectHolder<String> rowIdHolder = new ObjectHolder<>(null);
+        final AtomicReference<String> rowIdHolder = new AtomicReference<>(null);
 
         // convert each field/value to a column for the put, skip over nulls and arrays
         final Iterator<String> fieldNames = rootNode.getFieldNames();
         while (fieldNames.hasNext()) {
             final String fieldName = fieldNames.next();
-            final ObjectHolder<byte[]> fieldValueHolder = new ObjectHolder<>(null);
+            final AtomicReference<byte[]> fieldValueHolder = new AtomicReference<>(null);
 
             final JsonNode fieldNode = rootNode.get(fieldName);
             if (fieldNode.isNull()) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/SelectHiveQL.java
----------------------------------------------------------------------
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 77ded36..9c0ebef 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.HashSet;
 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.Relationship;
 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 class SelectHiveQL extends AbstractHiveQLProcessor {
 
         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 class SelectHiveQL extends AbstractHiveQLProcessor {
             });
 
             // 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)) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertAvroSchema.java
----------------------------------------------------------------------
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 ffcd653..a8244d2 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.Relationship;
 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.annotations.VisibleForTesting;
 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 class ConvertAvroSchema extends AbstractKiteProcessor {
         failureWriter.setCodec(CodecFactory.snappyCodec());
 
         try {
-            final LongHolder written = new LongHolder(0L);
+            final AtomicLong written = new AtomicLong(0L);
             final FailureTracker failures = new FailureTracker();
 
             final List<Record> badRecords = Lists.newLinkedList();

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertCSVToAvro.java
----------------------------------------------------------------------
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 43b33ff..de4130f 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.Relationship;
 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 org.kitesdk.data.spi.filesystem.CSVProperties;
 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 class ConvertCSVToAvro extends AbstractKiteProcessor {
             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);

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertJSONToAvro.java
----------------------------------------------------------------------
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 af120bf..6245362 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.ProcessSession;
 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 org.kitesdk.data.spi.filesystem.JSONFileReader;
 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 class ConvertJSONToAvro extends AbstractKiteProcessor {
         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);

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/image/ExtractImageMetadata.java
----------------------------------------------------------------------
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 1dd9241..06dd52b 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.Relationship;
 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 class ExtractImageMetadata extends AbstractProcessor {
         }
 
         final ComponentLog logger = this.getLogger();
-        final ObjectHolder<Metadata> value = new ObjectHolder<>(null);
+        final AtomicReference<Metadata> value = new AtomicReference<>(null);
         final Integer max = context.getProperty(MAX_NUMBER_OF_ATTRIBUTES).asInteger();
 
         try {

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java
----------------------------------------------------------------------
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 c6c4b15..0d7886d 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 @@ import java.util.stream.Collectors;
 
 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";

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java
----------------------------------------------------------------------
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 6e7c2f2..7445c5d 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.QueryResult;
 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 class VolatileProvenanceRepository implements ProvenanceEventRepository {
         @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<ProvenanceEventRecord> matchingRecords = new ArrayList<>(maxRecords);
             ringBuffer.forEach(new ForEachEvaluator<ProvenanceEventRecord>() {
                 @Override

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteProvenanceReportingTask.java
----------------------------------------------------------------------
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 268b734..a6eb662 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.ArrayList;
 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;

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/PutSolrContentStream.java
----------------------------------------------------------------------
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 ca16286..76a70d4 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.Set;
 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.Relationship;
 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 class PutSolrContentStream extends SolrProcessor {
             return;
         }
 
-        final ObjectHolder<Exception> error = new ObjectHolder<>(null);
-        final ObjectHolder<Exception> connectionError = new ObjectHolder<>(null);
+        final AtomicReference<Exception> error = new AtomicReference<>(null);
+        final AtomicReference<Exception> 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();

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-splunk-bundle/nifi-splunk-processors/src/main/java/org/apache/nifi/processors/splunk/PutSplunk.java
----------------------------------------------------------------------
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 c461b0f..9e4aa12 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.ByteArrayOutputStream;
 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.Arrays;
 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 @@ public class PutSplunk extends AbstractPutEventProcessor {
         // 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);
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java
----------------------------------------------------------------------
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 2f1ff43..21172e9 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.ProcessSession;
 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.HashMap;
 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<DocumentContext> contextHolder = new ObjectHolder<>(null);
+        final AtomicReference<DocumentContext> contextHolder = new AtomicReference<>(null);
         processSession.read(flowFile, new InputStreamCallback() {
             @Override
             public void process(InputStream in) throws IOException {

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/BinFiles.java
----------------------------------------------------------------------
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 3df4317..0000000
--- 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 <duration> <time unit> "
-                    + "where <duration> is a positive integer and time unit is one of seconds, minutes, hours")
-            .required(false)
-            .addValidator(StandardValidators.createTimePeriodValidator(1, TimeUnit.SECONDS, Integer.MAX_VALUE, TimeUnit.SECONDS))
-            .build();
-
-    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
-            .name("original")
-            .description("The FlowFiles that were used to create the bundle")
-            .build();
-    public static final Relationship REL_FAILURE = new Relationship.Builder()
-            .name("failure")
-            .description("If the bundle cannot be created, all FlowFiles that would have been used to created the bundle will be transferred to failure")
-            .build();
-
-    private final BinManager binManager = new BinManager();
-    private final Queue<Bin> readyBins = new LinkedBlockingQueue<>();
-
-    @OnStopped
-    public final void resetState() {
-        binManager.purge();
-
-        Bin bin;
-        while ((bin = readyBins.poll()) != null) {
-            for (final FlowFileSessionWrapper wrapper : bin.getContents()) {
-                wrapper.getSession().rollback();
-            }
-        }
-    }
-
-    /**
-     * Allows general pre-processing of a flow file before it is offered to a bin. This is called before getGroupId().
-     *
-     * @param context context
-     * @param session session
-     * @param flowFile flowFile
-     * @return The flow file, possibly altered
-     */
-    protected abstract FlowFile preprocessFlowFile(final ProcessContext context, final ProcessSession session, final FlowFile flowFile);
-
-    /**
-     * Returns a group ID representing a bin. This allows flow files to be binned into like groups.
-     *
-     * @param context context
-     * @param flowFile flowFile
-     * @return The appropriate group ID
-     */
-    protected abstract String getGroupId(final ProcessContext context, final FlowFile flowFile);
-
-    /**
-     * Performs any additional setup of the bin manager. Called during the OnScheduled phase.
-     *
-     * @param binManager The bin manager
-     * @param context context
-     */
-    protected abstract void setUpBinManager(BinManager binManager, ProcessContext context);
-
-    /**
-     * Processes a single bin. Implementing class is responsible for committing each session
-     *
-     * @param unmodifiableBin A reference to a single bin of flow file/session wrappers
-     * @param binContents A copy of the contents of the bin
-     * @param context The context
-     * @param session The session that created the bin
-     * @return Return true if the input bin was already committed. E.g., in case of a failure, the implementation may choose to transfer all binned files to Failure and commit their sessions. If
-     * false, the processBins() method will transfer the files to Original and commit the sessions
-     *
-     * @throws ProcessException if any problem arises while processing a bin of FlowFiles. All flow files in the bin will be transferred to failure and the ProcessSession provided by the 'session'
-     * argument rolled back
-     */
-    protected abstract boolean processBin(Bin unmodifiableBin, List<FlowFileSessionWrapper> binContents, ProcessContext context, ProcessSession session) throws ProcessException;
-
-    /**
-     * Allows additional custom validation to be done. This will be called from the parent's customValidation method.
-     *
-     * @param context The context
-     * @return Validation results indicating problems
-     */
-    protected Collection<ValidationResult> additionalCustomValidation(final ValidationContext context) {
-        return new ArrayList<>();
-    }
-
-    @Override
-    public final void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException {
-        final int flowFilesBinned = binFlowFiles(context, sessionFactory);
-        getLogger().debug("Binned {} FlowFiles", new Object[]{flowFilesBinned});
-
-        if (!isScheduled()) {
-            return;
-        }
-
-        final int binsMigrated = migrateBins(context);
-        final int binsProcessed = processBins(context, sessionFactory);
-        //If we accomplished nothing then let's yield
-        if (flowFilesBinned == 0 && binsMigrated == 0 && binsProcessed == 0) {
-            context.yield();
-        }
-    }
-
-    private int migrateBins(final ProcessContext context) {
-        int added = 0;
-        for (final Bin bin : binManager.removeReadyBins(true)) {
-            this.readyBins.add(bin);
-            added++;
-        }
-
-        // if we have created all of the bins that are allowed, go ahead and remove the oldest one. If we don't do
-        // this, then we will simply wait for it to expire because we can't get any more FlowFiles into the
-        // bins. So we may as well expire it now.
-        if (added == 0 && binManager.getBinCount() >= context.getProperty(MAX_BIN_COUNT).asInteger()) {
-            final Bin bin = binManager.removeOldestBin();
-            if (bin != null) {
-                added++;
-                this.readyBins.add(bin);
-            }
-        }
-        return added;
-    }
-
-    private int processBins(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
-        final Bin bin = readyBins.poll();
-        if (bin == null) {
-            return 0;
-        }
-
-        final List<Bin> bins = new ArrayList<>();
-        bins.add(bin);
-
-        final ComponentLog logger = getLogger();
-        final ProcessSession session = sessionFactory.createSession();
-
-        final List<FlowFileSessionWrapper> binCopy = new ArrayList<>(bin.getContents());
-
-        boolean binAlreadyCommitted = false;
-        try {
-            binAlreadyCommitted = this.processBin(bin, binCopy, context, session);
-        } catch (final ProcessException e) {
-            logger.error("Failed to process bundle of {} files due to {}", new Object[]{binCopy.size(), e});
-
-            for (final FlowFileSessionWrapper wrapper : binCopy) {
-                wrapper.getSession().transfer(wrapper.getFlowFile(), REL_FAILURE);
-                wrapper.getSession().commit();
-            }
-            session.rollback();
-            return 1;
-        }
-
-        // we first commit the bundle's session before the originals' sessions because if we are restarted or crash
-        // between commits, we favor data redundancy over data loss. Since we have no Distributed Transaction capability
-        // across multiple sessions, we cannot guarantee atomicity across the sessions
-        session.commit();
-        // If this bin's session has been committed, move on.
-        if (!binAlreadyCommitted) {
-            for (final FlowFileSessionWrapper wrapper : bin.getContents()) {
-                wrapper.getSession().transfer(wrapper.getFlowFile(), REL_ORIGINAL);
-                wrapper.getSession().commit();
-            }
-        }
-
-        return 1;
-    }
-
-    private int binFlowFiles(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
-        int flowFilesBinned = 0;
-        while (binManager.getBinCount() <= context.getProperty(MAX_BIN_COUNT).asInteger().intValue()) {
-            if (!isScheduled()) {
-                break;
-            }
-
-            final ProcessSession session = sessionFactory.createSession();
-            FlowFile flowFile = session.get();
-            if (flowFile == null) {
-                break;
-            }
-
-            flowFile = this.preprocessFlowFile(context, session, flowFile);
-
-            String groupId = this.getGroupId(context, flowFile);
-
-            final boolean binned = binManager.offer(groupId, flowFile, session);
-
-            // could not be added to a bin -- probably too large by itself, so create a separate bin for just this guy.
-            if (!binned) {
-                Bin bin = new Bin(0, Long.MAX_VALUE, 0, Integer.MAX_VALUE, null);
-                bin.offer(flowFile, session);
-                this.readyBins.add(bin);
-            }
-
-            flowFilesBinned++;
-        }
-
-        return flowFilesBinned;
-    }
-
-    @OnScheduled
-    public final void onScheduled(final ProcessContext context) throws IOException {
-        binManager.setMinimumSize(context.getProperty(MIN_SIZE).asDataSize(DataUnit.B).longValue());
-
-        if (context.getProperty(MAX_BIN_AGE).isSet()) {
-            binManager.setMaxBinAge(context.getProperty(MAX_BIN_AGE).asTimePeriod(TimeUnit.SECONDS).intValue());
-        } else {
-            binManager.setMaxBinAge(Integer.MAX_VALUE);
-        }
-
-        if (context.getProperty(MAX_SIZE).isSet()) {
-            binManager.setMaximumSize(context.getProperty(MAX_SIZE).asDataSize(DataUnit.B).longValue());
-        } else {
-            binManager.setMaximumSize(Long.MAX_VALUE);
-        }
-
-        binManager.setMinimumEntries(context.getProperty(MIN_ENTRIES).asInteger());
-
-        if (context.getProperty(MAX_ENTRIES).isSet()) {
-            binManager.setMaximumEntries(context.getProperty(MAX_ENTRIES).asInteger().intValue());
-        } else {
-            binManager.setMaximumEntries(Integer.MAX_VALUE);
-        }
-
-        this.setUpBinManager(binManager, context);
-    }
-
-    @Override
-    protected final Collection<ValidationResult> customValidate(final ValidationContext context) {
-        final List<ValidationResult> problems = new ArrayList<>(super.customValidate(context));
-
-        final long minBytes = context.getProperty(MIN_SIZE).asDataSize(DataUnit.B).longValue();
-        final Double maxBytes = context.getProperty(MAX_SIZE).asDataSize(DataUnit.B);
-
-        if (maxBytes != null && maxBytes.longValue() < minBytes) {
-            problems.add(
-                    new ValidationResult.Builder()
-                    .subject(MIN_SIZE.getName())
-                    .input(context.getProperty(MIN_SIZE).getValue())
-                    .valid(false)
-                    .explanation("Min Size must be less than or equal to Max Size")
-                    .build()
-            );
-        }
-
-        final Long min = context.getProperty(MIN_ENTRIES).asLong();
-        final Long max = context.getProperty(MAX_ENTRIES).asLong();
-
-        if (min != null && max != null) {
-            if (min > max) {
-                problems.add(
-                        new ValidationResult.Builder().subject(MIN_ENTRIES.getName())
-                        .input(context.getProperty(MIN_ENTRIES).getValue())
-                        .valid(false)
-                        .explanation("Min Entries must be less than or equal to Max Entries")
-                        .build()
-                );
-            }
-        }
-
-        Collection<ValidationResult> otherProblems = this.additionalCustomValidation(context);
-        if (otherProblems != null) {
-            problems.addAll(otherProblems);
-        }
-
-        return problems;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java
index e4e6cf4..1f4dd45 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java
@@ -28,6 +28,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.commons.compress.compressors.CompressorStreamFactory;
 import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
@@ -55,7 +56,6 @@ import org.apache.nifi.processor.io.StreamCallback;
 import org.apache.nifi.stream.io.BufferedInputStream;
 import org.apache.nifi.stream.io.BufferedOutputStream;
 import org.apache.nifi.stream.io.GZIPOutputStream;
-import org.apache.nifi.util.ObjectHolder;
 import org.apache.nifi.util.StopWatch;
 import org.tukaani.xz.LZMA2Options;
 import org.tukaani.xz.XZInputStream;
@@ -203,7 +203,7 @@ public class CompressContent extends AbstractProcessor {
         }
 
         final String compressionFormat = compressionFormatValue;
-        final ObjectHolder<String> mimeTypeRef = new ObjectHolder<>(null);
+        final AtomicReference<String> mimeTypeRef = new AtomicReference<>(null);
         final StopWatch stopWatch = new StopWatch(true);
 
         final String fileExtension;

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertJSONToSQL.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertJSONToSQL.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertJSONToSQL.java
index 3eb44cb..cbc301a 100755
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertJSONToSQL.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertJSONToSQL.java
@@ -33,6 +33,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.annotation.behavior.SideEffectFree;
@@ -57,7 +58,6 @@ import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.io.InputStreamCallback;
 import org.apache.nifi.processor.io.OutputStreamCallback;
 import org.apache.nifi.processor.util.StandardValidators;
-import org.apache.nifi.util.ObjectHolder;
 import org.codehaus.jackson.JsonNode;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.node.ArrayNode;
@@ -276,7 +276,7 @@ public class ConvertJSONToSQL extends AbstractProcessor {
 
         // Parse the JSON document
         final ObjectMapper mapper = new ObjectMapper();
-        final ObjectHolder<JsonNode> rootNodeRef = new ObjectHolder<>(null);
+        final AtomicReference<JsonNode> rootNodeRef = new AtomicReference<>(null);
         try {
             session.read(flowFile, new InputStreamCallback() {
                 @Override

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
index 18259d1..813a07d 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java
@@ -52,12 +52,12 @@ import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.io.OutputStreamCallback;
 import org.apache.nifi.stream.io.BufferedOutputStream;
-import org.apache.nifi.util.ObjectHolder;
 
 import com.jayway.jsonpath.DocumentContext;
 import com.jayway.jsonpath.InvalidJsonException;
 import com.jayway.jsonpath.JsonPath;
 import com.jayway.jsonpath.PathNotFoundException;
+import java.util.concurrent.atomic.AtomicReference;
 
 @EventDriven
 @SideEffectFree
@@ -277,7 +277,7 @@ public class EvaluateJsonPath extends AbstractJsonPathProcessor {
             final JsonPath jsonPathExp = attributeJsonPathEntry.getValue();
             final String pathNotFound = processContext.getProperty(PATH_NOT_FOUND).getValue();
 
-            final ObjectHolder<Object> resultHolder = new ObjectHolder<>(null);
+            final AtomicReference<Object> resultHolder = new AtomicReference<>(null);
             try {
                 final Object result = documentContext.read(jsonPathExp);
                 if (returnType.equals(RETURN_TYPE_SCALAR) && !isJsonScalar(result)) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java
index 10d9af9..4ff7e0e 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java
@@ -75,7 +75,6 @@ import org.apache.nifi.processor.io.InputStreamCallback;
 import org.apache.nifi.processor.io.OutputStreamCallback;
 import org.apache.nifi.stream.io.BufferedInputStream;
 import org.apache.nifi.stream.io.BufferedOutputStream;
-import org.apache.nifi.util.ObjectHolder;
 import org.xml.sax.InputSource;
 
 import net.sf.saxon.lib.NamespaceConstant;
@@ -271,8 +270,8 @@ public class EvaluateXPath extends AbstractProcessor {
 
         flowFileLoop:
         for (FlowFile flowFile : flowFiles) {
-            final ObjectHolder<Throwable> error = new ObjectHolder<>(null);
-            final ObjectHolder<Source> sourceRef = new ObjectHolder<>(null);
+            final AtomicReference<Throwable> error = new AtomicReference<>(null);
+            final AtomicReference<Source> sourceRef = new AtomicReference<>(null);
 
             session.read(flowFile, new InputStreamCallback() {
                 @Override
@@ -402,7 +401,7 @@ public class EvaluateXPath extends AbstractProcessor {
 
         final ComponentLog logger = getLogger();
 
-        final ObjectHolder<TransformerException> error = new ObjectHolder<>(null);
+        final AtomicReference<TransformerException> error = new AtomicReference<>(null);
         transformer.setErrorListener(new ErrorListener() {
             @Override
             public void warning(final TransformerException exception) throws TransformerException {

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java
index 51a7db0..b8ff2eb 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java
@@ -29,6 +29,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
 
 import javax.xml.parsers.DocumentBuilderFactory;
 import javax.xml.transform.OutputKeys;
@@ -66,7 +67,6 @@ import org.apache.nifi.processor.io.OutputStreamCallback;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.stream.io.BufferedInputStream;
 import org.apache.nifi.stream.io.BufferedOutputStream;
-import org.apache.nifi.util.ObjectHolder;
 import org.w3c.dom.Document;
 import org.xml.sax.InputSource;
 
@@ -264,8 +264,8 @@ public class EvaluateXQuery extends AbstractProcessor {
                 return;
             }
 
-            final ObjectHolder<Throwable> error = new ObjectHolder<>(null);
-            final ObjectHolder<XdmNode> sourceRef = new ObjectHolder<>(null);
+            final AtomicReference<Throwable> error = new AtomicReference<>(null);
+            final AtomicReference<XdmNode> sourceRef = new AtomicReference<>(null);
 
             session.read(flowFile, new InputStreamCallback() {
                 @Override

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java
index 95893d8..1c514b4 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java
@@ -29,6 +29,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.nifi.annotation.behavior.EventDriven;
@@ -51,7 +52,6 @@ import org.apache.nifi.processor.io.InputStreamCallback;
 import org.apache.nifi.processor.io.OutputStreamCallback;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.processors.standard.util.JdbcCommon;
-import org.apache.nifi.util.LongHolder;
 import org.apache.nifi.util.StopWatch;
 
 @EventDriven
@@ -180,7 +180,7 @@ public class ExecuteSQL extends AbstractProcessor {
         try (final Connection con = dbcpService.getConnection();
             final Statement st = con.createStatement()) {
             st.setQueryTimeout(queryTimeout); // timeout in seconds
-            final LongHolder nrOfRows = new LongHolder(0L);
+            final AtomicLong nrOfRows = new AtomicLong(0L);
             if (fileToProcess == null) {
                 fileToProcess = session.create();
             }
@@ -198,7 +198,7 @@ public class ExecuteSQL extends AbstractProcessor {
             });
 
             // 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()});

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashContent.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashContent.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashContent.java
index cea3803..ade251c 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashContent.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashContent.java
@@ -27,6 +27,7 @@ import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.nifi.annotation.behavior.EventDriven;
 import org.apache.nifi.annotation.behavior.InputRequirement;
@@ -48,7 +49,6 @@ import org.apache.nifi.processor.io.InputStreamCallback;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.stream.io.NullOutputStream;
 import org.apache.nifi.stream.io.StreamUtils;
-import org.apache.nifi.util.ObjectHolder;
 
 @EventDriven
 @SupportsBatching
@@ -129,7 +129,7 @@ public class HashContent extends AbstractProcessor {
             return;
         }
 
-        final ObjectHolder<String> hashValueHolder = new ObjectHolder<>(null);
+        final AtomicReference<String> hashValueHolder = new AtomicReference<>(null);
 
         try {
             session.read(flowFile, new InputStreamCallback() {

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/IdentifyMimeType.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/IdentifyMimeType.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/IdentifyMimeType.java
index f4189c3..5965cfd 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/IdentifyMimeType.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/IdentifyMimeType.java
@@ -22,6 +22,7 @@ import java.io.InputStream;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.nifi.annotation.behavior.EventDriven;
 import org.apache.nifi.annotation.behavior.InputRequirement;
@@ -40,7 +41,6 @@ import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.ProcessorInitializationContext;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.io.InputStreamCallback;
-import org.apache.nifi.util.ObjectHolder;
 import org.apache.tika.config.TikaConfig;
 import org.apache.tika.detect.Detector;
 import org.apache.tika.io.TikaInputStream;
@@ -117,7 +117,7 @@ public class IdentifyMimeType extends AbstractProcessor {
         }
 
         final ComponentLog logger = getLogger();
-        final ObjectHolder<String> mimeTypeRef = new ObjectHolder<>(null);
+        final AtomicReference<String> mimeTypeRef = new AtomicReference<>(null);
         final String filename = flowFile.getAttribute(CoreAttributes.FILENAME.key());
 
         session.read(flowFile, new InputStreamCallback() {

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/JmsConsumer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/JmsConsumer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/JmsConsumer.java
index cf968a6..71e2500 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/JmsConsumer.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/JmsConsumer.java
@@ -41,6 +41,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import javax.jms.JMSException;
 import javax.jms.MapMessage;
@@ -60,7 +61,6 @@ import org.apache.nifi.processors.standard.util.JmsFactory;
 import org.apache.nifi.processors.standard.util.JmsProcessingSummary;
 import org.apache.nifi.processors.standard.util.WrappedMessageConsumer;
 import org.apache.nifi.stream.io.BufferedOutputStream;
-import org.apache.nifi.util.IntegerHolder;
 import org.apache.nifi.util.StopWatch;
 
 public abstract class JmsConsumer extends AbstractProcessor {
@@ -181,7 +181,7 @@ public abstract class JmsConsumer extends AbstractProcessor {
             throws Exception {
 
         // Currently not very useful, because always one Message == one FlowFile
-        final IntegerHolder msgsThisFlowFile = new IntegerHolder(1);
+        final AtomicInteger msgsThisFlowFile = new AtomicInteger(1);
 
         FlowFile flowFile = session.create();
         try {

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
index ce216d7..545559e 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
@@ -34,6 +34,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeMap;
+import java.util.concurrent.atomic.AtomicReference;
 import java.util.regex.Pattern;
 import java.util.zip.ZipEntry;
 import java.util.zip.ZipOutputStream;
@@ -85,7 +86,6 @@ import org.apache.nifi.util.FlowFilePackager;
 import org.apache.nifi.util.FlowFilePackagerV1;
 import org.apache.nifi.util.FlowFilePackagerV2;
 import org.apache.nifi.util.FlowFilePackagerV3;
-import org.apache.nifi.util.ObjectHolder;
 
 @SideEffectFree
 @TriggerWhenEmpty
@@ -557,7 +557,7 @@ public class MergeContent extends BinFiles {
             }
 
             FlowFile bundle = session.create(parentFlowFiles);
-            final ObjectHolder<String> bundleMimeTypeRef = new ObjectHolder<>(null);
+            final AtomicReference<String> bundleMimeTypeRef = new AtomicReference<>(null);
             bundle = session.write(bundle, new OutputStreamCallback() {
                 @Override
                 public void process(final OutputStream out) throws IOException {
@@ -884,8 +884,8 @@ public class MergeContent extends BinFiles {
         public FlowFile merge(ProcessContext context, final ProcessSession session, final List<FlowFileSessionWrapper> wrappers) {
 
             final Map<String, byte[]> metadata = new TreeMap<>();
-            final ObjectHolder<Schema> schema = new ObjectHolder<>(null);
-            final ObjectHolder<String> inputCodec = new ObjectHolder<>(null);
+            final AtomicReference<Schema> schema = new AtomicReference<>(null);
+            final AtomicReference<String> inputCodec = new AtomicReference<>(null);
             final DataFileWriter<GenericRecord> writer = new DataFileWriter<>(new GenericDatumWriter<GenericRecord>());
 
             // we don't pass the parents to the #create method because the parents belong to different sessions

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java
index d3713a1..2239995 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java
@@ -117,7 +117,6 @@ import org.apache.nifi.util.FlowFilePackagerV1;
 import org.apache.nifi.util.FlowFilePackagerV2;
 import org.apache.nifi.util.FlowFilePackagerV3;
 import org.apache.nifi.util.FormatUtils;
-import org.apache.nifi.util.ObjectHolder;
 import org.apache.nifi.util.StopWatch;
 import org.apache.nifi.util.StringUtils;
 import com.sun.jersey.api.client.ClientResponse.Status;
@@ -466,7 +465,7 @@ public class PostHTTP extends AbstractProcessor {
         CloseableHttpClient client = null;
         final String transactionId = UUID.randomUUID().toString();
 
-        final ObjectHolder<String> dnHolder = new ObjectHolder<>("none");
+        final AtomicReference<String> dnHolder = new AtomicReference<>("none");
         while (true) {
             FlowFile flowFile = session.get();
             if (flowFile == null) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFileTransfer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFileTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFileTransfer.java
index f41941d..5b9f5b4 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFileTransfer.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFileTransfer.java
@@ -30,7 +30,6 @@ import org.apache.nifi.processor.io.InputStreamCallback;
 import org.apache.nifi.processors.standard.util.FileInfo;
 import org.apache.nifi.processors.standard.util.FileTransfer;
 import org.apache.nifi.processors.standard.util.SFTPTransfer;
-import org.apache.nifi.util.ObjectHolder;
 import org.apache.nifi.util.StopWatch;
 
 import java.io.File;
@@ -40,6 +39,7 @@ import java.util.Collections;
 import java.util.HashSet;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
  * Base class for PutFTP & PutSFTP
@@ -123,7 +123,7 @@ public abstract class PutFileTransfer<T extends FileTransfer> extends AbstractPr
 
                     beforePut(flowFile, context, transfer);
                     final FlowFile flowFileToTransfer = flowFile;
-                    final ObjectHolder<String> fullPathRef = new ObjectHolder<>(null);
+                    final AtomicReference<String> fullPathRef = new AtomicReference<>(null);
                     session.read(flowFile, new InputStreamCallback() {
                         @Override
                         public void process(final InputStream in) throws IOException {

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSyslog.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSyslog.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSyslog.java
index 373e402..412d8ab 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSyslog.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSyslog.java
@@ -40,7 +40,6 @@ import org.apache.nifi.processor.util.put.sender.SSLSocketChannelSender;
 import org.apache.nifi.processor.util.put.sender.SocketChannelSender;
 import org.apache.nifi.processors.standard.syslog.SyslogParser;
 import org.apache.nifi.ssl.SSLContextService;
-import org.apache.nifi.util.ObjectHolder;
 import org.apache.nifi.util.StopWatch;
 
 import javax.net.ssl.SSLContext;
@@ -55,6 +54,7 @@ import java.util.Set;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -321,7 +321,7 @@ public class PutSyslog extends AbstractSyslogProcessor {
         final String port = context.getProperty(PORT).getValue();
         final String host = context.getProperty(HOSTNAME).getValue();
         final String transitUri = new StringBuilder().append(protocol).append("://").append(host).append(":").append(port).toString();
-        final ObjectHolder<IOException> exceptionHolder = new ObjectHolder<>(null);
+        final AtomicReference<IOException> exceptionHolder = new AtomicReference<>(null);
         final Charset charSet = Charset.forName(context.getProperty(CHARSET).getValue());
 
         try {

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryDatabaseTable.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryDatabaseTable.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryDatabaseTable.java
index ec81446..e749696 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryDatabaseTable.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryDatabaseTable.java
@@ -41,7 +41,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.processors.standard.util.JdbcCommon;
-import org.apache.nifi.util.LongHolder;
 import org.apache.nifi.util.StopWatch;
 
 import java.io.IOException;
@@ -93,6 +92,7 @@ import static java.sql.Types.TIMESTAMP;
 import static java.sql.Types.TINYINT;
 import static java.sql.Types.VARBINARY;
 import static java.sql.Types.VARCHAR;
+import java.util.concurrent.atomic.AtomicLong;
 
 @EventDriven
 @InputRequirement(Requirement.INPUT_FORBIDDEN)
@@ -296,7 +296,7 @@ public class QueryDatabaseTable extends AbstractSessionFactoryProcessor {
             final Integer queryTimeout = context.getProperty(QUERY_TIMEOUT).asTimePeriod(TimeUnit.SECONDS).intValue();
             st.setQueryTimeout(queryTimeout); // timeout in seconds
 
-            final LongHolder nrOfRows = new LongHolder(0L);
+            final AtomicLong nrOfRows = new AtomicLong(0L);
 
             fileToProcess = session.create();
             fileToProcess = session.write(fileToProcess, new OutputStreamCallback() {
@@ -317,7 +317,7 @@ public class QueryDatabaseTable extends AbstractSessionFactoryProcessor {
 
             if (nrOfRows.get() > 0) {
                 // 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()});

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java
index 1bf235b..64877f2 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java
@@ -26,6 +26,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.regex.Pattern;
 
@@ -51,7 +52,6 @@ import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.io.InputStreamCallback;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.stream.io.StreamUtils;
-import org.apache.nifi.util.IntegerHolder;
 
 @EventDriven
 @SideEffectFree
@@ -182,7 +182,7 @@ public class RouteOnContent extends AbstractProcessor {
             final Set<Relationship> destinations = new HashSet<>();
             flowFileDestinationMap.put(flowFile, destinations);
 
-            final IntegerHolder bufferedByteCount = new IntegerHolder(0);
+            final AtomicInteger bufferedByteCount = new AtomicInteger(0);
             session.read(flowFile, new InputStreamCallback() {
                 @Override
                 public void process(final InputStream in) throws IOException {

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanContent.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanContent.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanContent.java
index 59d1cbd..1f635f9 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanContent.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanContent.java
@@ -54,7 +54,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.stream.io.BufferedInputStream;
-import org.apache.nifi.util.ObjectHolder;
 import org.apache.nifi.util.file.monitor.LastModifiedMonitor;
 import org.apache.nifi.util.file.monitor.SynchronousFileWatcher;
 import org.apache.nifi.util.search.Search;
@@ -219,7 +218,7 @@ public class ScanContent extends AbstractProcessor {
         }
 
         final Search<byte[]> finalSearch = search;
-        final ObjectHolder<SearchTerm<byte[]>> termRef = new ObjectHolder<>(null);
+        final AtomicReference<SearchTerm<byte[]>> termRef = new AtomicReference<>(null);
         termRef.set(null);
 
         session.read(flowFile, new InputStreamCallback() {

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java
index 0f1abfd..81c06f8 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java
@@ -32,6 +32,9 @@ import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.nifi.annotation.behavior.EventDriven;
 import org.apache.nifi.annotation.behavior.InputRequirement;
@@ -63,9 +66,6 @@ import org.apache.nifi.stream.io.BufferedOutputStream;
 import org.apache.nifi.stream.io.ByteArrayOutputStream;
 import org.apache.nifi.stream.io.ByteCountingInputStream;
 import org.apache.nifi.stream.io.ByteCountingOutputStream;
-import org.apache.nifi.util.IntegerHolder;
-import org.apache.nifi.util.LongHolder;
-import org.apache.nifi.util.ObjectHolder;
 
 @EventDriven
 @SideEffectFree
@@ -412,7 +412,7 @@ public class SplitText extends AbstractProcessor {
         final String headerMarker = context.getProperty(HEADER_MARKER).getValue();
         final boolean includeLineDelimiter = !context.getProperty(REMOVE_TRAILING_NEWLINES).asBoolean();
 
-        final ObjectHolder<String> errorMessage = new ObjectHolder<>(null);
+        final AtomicReference<String> errorMessage = new AtomicReference<>(null);
         final ArrayList<SplitInfo> splitInfos = new ArrayList<>();
 
         final long startNanos = System.nanoTime();
@@ -479,8 +479,8 @@ public class SplitText extends AbstractProcessor {
                         if (headerInfoLineCount > 0) {
                             // if we have header lines, create a new FlowFile, copy the header lines to that file,
                             // and then start copying lines
-                            final IntegerHolder linesCopied = new IntegerHolder(0);
-                            final LongHolder bytesCopied = new LongHolder(0L);
+                            final AtomicInteger linesCopied = new AtomicInteger(0);
+                            final AtomicLong bytesCopied = new AtomicLong(0L);
                             FlowFile splitFile = session.create(flowFile);
                             try {
                                 splitFile = session.write(splitFile, new OutputStreamCallback() {

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitXml.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitXml.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitXml.java
index 055ffd8..4764ea8 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitXml.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitXml.java
@@ -27,6 +27,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.TreeMap;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import javax.xml.parsers.ParserConfigurationException;
 import javax.xml.parsers.SAXParser;
@@ -53,7 +54,6 @@ import org.apache.nifi.processor.io.OutputStreamCallback;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.processors.standard.util.XmlElementNotifier;
 import org.apache.nifi.stream.io.BufferedInputStream;
-import org.apache.nifi.util.BooleanHolder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.xml.sax.Attributes;
@@ -160,7 +160,7 @@ public class SplitXml extends AbstractProcessor {
             }
         }, depth);
 
-        final BooleanHolder failed = new BooleanHolder(false);
+        final AtomicBoolean failed = new AtomicBoolean(false);
         session.read(original, new InputStreamCallback() {
             @Override
             public void process(final InputStream rawIn) throws IOException {

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java
index b3ece4b..1ebf736 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java
@@ -39,6 +39,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
 import java.util.zip.CRC32;
 import java.util.zip.CheckedInputStream;
 import java.util.zip.Checksum;
@@ -68,7 +69,6 @@ import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.stream.io.ByteArrayOutputStream;
 import org.apache.nifi.stream.io.NullOutputStream;
 import org.apache.nifi.stream.io.StreamUtils;
-import org.apache.nifi.util.LongHolder;
 
 // note: it is important that this Processor is not marked as @SupportsBatching because the session commits must complete before persisting state locally; otherwise, data loss may occur
 @TriggerSerially
@@ -463,7 +463,7 @@ public class TailFile extends AbstractProcessor {
         FlowFile flowFile = session.create();
 
         final FileChannel fileReader = reader;
-        final LongHolder positionHolder = new LongHolder(position);
+        final AtomicLong positionHolder = new AtomicLong(position);
         flowFile = session.write(flowFile, new OutputStreamCallback() {
             @Override
             public void process(final OutputStream rawOut) throws IOException {


[3/6] nifi git commit: NIFI-1307 removed deprecation indication for getId and provided better API documentation

Posted by ma...@apache.org.
NIFI-1307 removed deprecation indication for getId and provided better API documentation


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/4c9d4655
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/4c9d4655
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/4c9d4655

Branch: refs/heads/master
Commit: 4c9d4655a84ab4d308fcd7234fbdd8942e71e403
Parents: 6e5e4cf
Author: joewitt <jo...@apache.org>
Authored: Wed Jul 13 18:42:10 2016 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Thu Jul 14 09:32:34 2016 -0400

----------------------------------------------------------------------
 .../main/java/org/apache/nifi/flowfile/FlowFile.java    | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/4c9d4655/nifi-api/src/main/java/org/apache/nifi/flowfile/FlowFile.java
----------------------------------------------------------------------
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 43fbf20..9063a8f 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 @@ import java.util.Set;
 public interface FlowFile extends Comparable<FlowFile> {
 
     /**
-     * @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();
 
     /**


[2/6] nifi git commit: NIFI-1157 resolved deprecated nifi-api items and ripple effects

Posted by ma...@apache.org.
NIFI-1157 resolved deprecated nifi-api items and ripple effects


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/961be21a
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/961be21a
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/961be21a

Branch: refs/heads/master
Commit: 961be21a3899191f49279295564bc22b8787b11a
Parents: 4c9d465
Author: joewitt <jo...@apache.org>
Authored: Wed Jul 13 19:36:03 2016 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Thu Jul 14 09:32:34 2016 -0400

----------------------------------------------------------------------
 .../controller/AbstractControllerService.java   |   4 +-
 .../controller/annotation/OnConfigured.java     |  44 ------
 .../java/org/apache/nifi/flowfile/FlowFile.java |  20 ---
 .../annotation/CapabilityDescription.java       |  42 -----
 .../nifi/processor/annotation/EventDriven.java  |  51 -------
 .../nifi/processor/annotation/OnAdded.java      |  43 ------
 .../nifi/processor/annotation/OnRemoved.java    |  44 ------
 .../nifi/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 ------
 .../processor/annotation/SupportsBatching.java  |  53 -------
 .../apache/nifi/processor/annotation/Tags.java  |  43 ------
 .../processor/annotation/TriggerSerially.java   |  41 -----
 .../TriggerWhenAnyDestinationAvailable.java     |  43 ------
 .../processor/annotation/TriggerWhenEmpty.java  |  43 ------
 .../nifi/provenance/ProvenanceEventBuilder.java |  10 --
 .../nifi/provenance/ProvenanceEventRecord.java  |  14 --
 .../nifi/provenance/lineage/LineageNode.java    |   7 -
 .../apache/nifi/reporting/ReportingTask.java    |   4 +-
 .../provenance/PlaceholderProvenanceEvent.java  |   5 -
 .../StandardProvenanceEventRecord.java          |  26 +---
 .../nifi/provenance/lineage/EventNode.java      |  11 --
 .../nifi/provenance/lineage/FlowFileNode.java   |   7 -
 .../expression/language/TestQuery.java          |   1 -
 .../java/org/apache/nifi/util/MockFlowFile.java |   9 --
 .../nifi/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 +++++++++----------
 .../nifi/controller/FileSystemSwapManager.java  |  21 +--
 .../apache/nifi/controller/FlowController.java  |   8 +-
 .../nifi/controller/StandardProcessorNode.java  |  33 +---
 .../reporting/AbstractReportingTaskNode.java    |  25 +--
 .../repository/StandardFlowFileRecord.java      |  29 ----
 .../repository/StandardProcessSession.java      |   7 -
 .../WriteAheadFlowFileRepository.java           |  30 ++--
 .../scheduling/EventDrivenSchedulingAgent.java  |   2 +-
 .../service/StandardControllerServiceNode.java  |  21 +--
 .../tasks/ContinuallyRunConnectableTask.java    |   2 +-
 .../controller/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 -
 .../nifi/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(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-api/src/main/java/org/apache/nifi/controller/AbstractControllerService.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 83d3301..15d30c3 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 abstract class AbstractControllerService extends AbstractConfigurableComp
         return identifier;
     }
 
-    @OnConfigured
+    @OnConfigurationRestored
     public void onConfigurationChange(final ConfigurationContext context) {
         this.configContext = context;
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-api/src/main/java/org/apache/nifi/controller/annotation/OnConfigured.java
----------------------------------------------------------------------
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 d56b969..0000000
--- 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 {
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-api/src/main/java/org/apache/nifi/flowfile/FlowFile.java
----------------------------------------------------------------------
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 9063a8f..0f4d2d0 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;
 
 /**
  * <p>
@@ -87,25 +86,6 @@ public interface FlowFile extends Comparable<FlowFile> {
     long getQueueDateIndex();
 
     /**
-     * <p>
-     * If a FlowFile is derived from multiple "parent" FlowFiles, all of the
-     * parents' Lineage Identifiers will be in the set.
-     * </p>
-     *
-     * @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<String> getLineageIdentifiers();
-
-    /**
      * @return true if flow file is currently penalized; false otherwise;
      */
     boolean isPenalized();

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-api/src/main/java/org/apache/nifi/processor/annotation/CapabilityDescription.java
----------------------------------------------------------------------
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 f966f89..0000000
--- 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();
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-api/src/main/java/org/apache/nifi/processor/annotation/EventDriven.java
----------------------------------------------------------------------
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 0f412ca..0000000
--- 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;
-
-/**
- * <p>
- * 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.
- * </p>
- *
- * <p>
- * 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.
- * </p>
- *
- * @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 {
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnAdded.java
----------------------------------------------------------------------
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 ff0b75f..0000000
--- 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 {
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnRemoved.java
----------------------------------------------------------------------
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 740e9f8..0000000
--- 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 {
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnScheduled.java
----------------------------------------------------------------------
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 2fec39c..0000000
--- 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 {
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnShutdown.java
----------------------------------------------------------------------
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 930a9df..0000000
--- 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 {
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnStopped.java
----------------------------------------------------------------------
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 4fbaf95..0000000
--- 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;
-
-/**
- * <p>
- * 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 <code>onTrigger</code> method.
- * </p>
- *
- * <p>
- * 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.
- * </p>
- *
- * <p>
- * To indicate that a method should be called immediately when a processor is no
- * longer scheduled to run, see the {@link OnUnscheduled} annotation.
- * </p>
- *
- * @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 {
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-api/src/main/java/org/apache/nifi/processor/annotation/OnUnscheduled.java
----------------------------------------------------------------------
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 7b4c1f6..0000000
--- 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;
-
-/**
- * <p>
- * 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.
- * </p>
- *
- * 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 {
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-api/src/main/java/org/apache/nifi/processor/annotation/SideEffectFree.java
----------------------------------------------------------------------
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 6c65caa..0000000
--- 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
- * <code>ProcessSession</code> 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 {
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-api/src/main/java/org/apache/nifi/processor/annotation/SupportsBatching.java
----------------------------------------------------------------------
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 e39e212..0000000
--- 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 {
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-api/src/main/java/org/apache/nifi/processor/annotation/Tags.java
----------------------------------------------------------------------
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 81428d7..0000000
--- 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();
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-api/src/main/java/org/apache/nifi/processor/annotation/TriggerSerially.java
----------------------------------------------------------------------
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 6e80cef..0000000
--- 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 {
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-api/src/main/java/org/apache/nifi/processor/annotation/TriggerWhenAnyDestinationAvailable.java
----------------------------------------------------------------------
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 6e01f6c..0000000
--- 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 {
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-api/src/main/java/org/apache/nifi/processor/annotation/TriggerWhenEmpty.java
----------------------------------------------------------------------
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 d068a9e..0000000
--- 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 {
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventBuilder.java
----------------------------------------------------------------------
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 0ffccd5..d0e20d1 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;
@@ -51,15 +50,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<String> lineageIdentifiers);
-
-    /**
      * Sets the Content Claim that the FlowFile was previously associated with
      * before this event occurred.
      *

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventRecord.java
----------------------------------------------------------------------
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 53b9c3e..eaa3464 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 @@ package org.apache.nifi.provenance;
 
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 /**
  * Describes an event that happened to a FlowFile.
@@ -49,19 +48,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<String> getLineageIdentifiers();
-
-    /**
      * @return the size of the FlowFile to which this Event is associated
      */
     long getFileSize();

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/LineageNode.java
----------------------------------------------------------------------
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 56e865f..5d0956e 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
@@ -19,13 +19,6 @@ package org.apache.nifi.provenance.lineage;
 public interface LineageNode {
 
     /**
-     * @return the identifier of the Clustered NiFi Node that generated the
-     * event
-     */
-    @Deprecated
-    String getClusterNodeIdentifier();
-
-    /**
      * @return the type of the LineageNode
      */
     LineageNodeType getNodeType();

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-api/src/main/java/org/apache/nifi/reporting/ReportingTask.java
----------------------------------------------------------------------
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 3de9b93..780b79e 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 @@ import org.apache.nifi.controller.annotation.OnConfigured;
  *
  * <p>
  * 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

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/PlaceholderProvenanceEvent.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 bc40302..083b13e 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
@@ -60,11 +60,6 @@ public class PlaceholderProvenanceEvent implements ProvenanceEventRecord {
     }
 
     @Override
-    public Set<String> getLineageIdentifiers() {
-        return Collections.emptySet();
-    }
-
-    @Override
     public long getFileSize() {
         return -1L;
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java
----------------------------------------------------------------------
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 b504b04..7b0c91f 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.ArrayList;
 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 @@ import org.apache.nifi.processor.Relationship;
 
 /**
  * Holder for provenance relevant information
- * <p/>
  */
 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<String> lineageIdentifiers;
     private final String componentId;
     private final String componentType;
     private final String transitUri;
@@ -91,7 +87,6 @@ public final class StandardProvenanceEventRecord implements ProvenanceEventRecor
         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 @@ public final class StandardProvenanceEventRecord implements ProvenanceEventRecor
         updatedAttributes = builder.updatedAttributes == null ? Collections.<String, String>emptyMap() : Collections.unmodifiableMap(builder.updatedAttributes);
 
         sourceQueueIdentifier = builder.sourceQueueIdentifier;
-
+    
     }
 
     public String getStorageFilename() {
@@ -135,11 +130,6 @@ public final class StandardProvenanceEventRecord implements ProvenanceEventRecor
     }
 
     @Override
-    public Set<String> getLineageIdentifiers() {
-        return lineageIdentifiers;
-    }
-
-    @Override
     public long getLineageStartDate() {
         return lineageStartDate;
     }
@@ -414,7 +404,6 @@ public final class StandardProvenanceEventRecord implements ProvenanceEventRecor
         private long eventTime = System.currentTimeMillis();
         private long entryDate;
         private long lineageStartDate;
-        private Set<String> lineageIdentifiers = new HashSet<>();
         private ProvenanceEventType eventType = null;
         private String componentId = null;
         private String componentType = null;
@@ -453,7 +442,6 @@ public final class StandardProvenanceEventRecord implements ProvenanceEventRecor
             eventTime = event.getEventTime();
             entryDate = event.getFlowFileEntryDate();
             lineageStartDate = event.getLineageStartDate();
-            lineageIdentifiers = event.getLineageIdentifiers();
             eventType = event.getEventType();
             componentId = event.getComponentId();
             componentType = event.getComponentType();
@@ -499,12 +487,6 @@ public final class StandardProvenanceEventRecord implements ProvenanceEventRecor
         }
 
         @Override
-        public Builder setLineageIdentifiers(final Set<String> lineageIdentifiers) {
-            this.lineageIdentifiers = lineageIdentifiers;
-            return this;
-        }
-
-        @Override
         public Builder setAttributes(final Map<String, String> previousAttributes, final Map<String, String> updatedAttributes) {
             this.previousAttributes = previousAttributes;
             this.updatedAttributes = updatedAttributes;
@@ -541,11 +523,6 @@ public final class StandardProvenanceEventRecord implements ProvenanceEventRecor
             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 final class StandardProvenanceEventRecord implements ProvenanceEventRecor
         @Override
         public ProvenanceEventBuilder fromFlowFile(final FlowFile flowFile) {
             setFlowFileEntryDate(flowFile.getEntryDate());
-            setLineageIdentifiers(flowFile.getLineageIdentifiers());
             setLineageStartDate(flowFile.getLineageStartDate());
             setAttributes(Collections.<String, String>emptyMap(), flowFile.getAttributes());
             uuid = flowFile.getAttribute(CoreAttributes.UUID.key());

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/EventNode.java
----------------------------------------------------------------------
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 1f8d1dc..de516cb 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 class EventNode implements ProvenanceEventLineageNode {
         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;

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/FlowFileNode.java
----------------------------------------------------------------------
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 94e7661..7881338 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 class FlowFileNode implements LineageNode {
         return creationTime;
     }
 
-    @Deprecated
-    @Override
-    public String getClusterNodeIdentifier() {
-        return clusterNodeIdentifier;
-    }
-
     @Override
     public LineageNodeType getNodeType() {
         return LineageNodeType.FLOWFILE_NODE;

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java
----------------------------------------------------------------------
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 05bd017..0ede6cc 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 @@ public class TestQuery {
         Mockito.when(mockFlowFile.getId()).thenReturn(1L);
         Mockito.when(mockFlowFile.getEntryDate()).thenReturn(System.currentTimeMillis());
         Mockito.when(mockFlowFile.getSize()).thenReturn(1L);
-        Mockito.when(mockFlowFile.getLineageIdentifiers()).thenReturn(new HashSet<String>());
         Mockito.when(mockFlowFile.getLineageStartDate()).thenReturn(System.currentTimeMillis());
         return Query.evaluateExpressions(queryString, mockFlowFile);
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-mock/src/main/java/org/apache/nifi/util/MockFlowFile.java
----------------------------------------------------------------------
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 049c65b..02806b0 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<String> lineageIdentifiers = new HashSet<>();
     private final long creationTime;
     private boolean penalized = false;
 
@@ -60,7 +59,6 @@ public class MockFlowFile implements FlowFileRecord {
 
         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 class MockFlowFile implements FlowFileRecord {
         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() {
@@ -82,11 +78,6 @@ public class MockFlowFile implements FlowFileRecord {
     }
 
     @Override
-    public Set<String> getLineageIdentifiers() {
-        return lineageIdentifiers;
-    }
-
-    @Override
     public long getLineageStartDate() {
         return entryDate;
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
----------------------------------------------------------------------
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 8979353..7d49f0f 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 static java.util.Objects.requireNonNull;
 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<Class<? extends Annotation>> deprecatedTypeAnnotations = new HashSet<>();
-    private static final Set<Class<? extends Annotation>> deprecatedMethodAnnotations = new HashSet<>();
     private final Map<String, MockComponentLog> 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<? extends Annotation> annotationClass : deprecatedTypeAnnotations) {
-            if (processor.getClass().isAnnotationPresent(annotationClass)) {
-                Assert.fail("Processor is using deprecated Annotation " + annotationClass.getCanonicalName());
-            }
-        }
-
-        for (final Class<? extends Annotation> 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);

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-mock/src/test/java/org/apache/nifi/util/TestStandardProcessorTestRunner.java
----------------------------------------------------------------------
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 342b016..0e41e89 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.ProcessSession;
 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 class TestStandardProcessorTestRunner {
         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 class TestStandardProcessorTestRunner {
         }
     }
 
-    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;

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/util/FlowFileEvent.java
----------------------------------------------------------------------
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 c552d4d..d682946 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 org.apache.nifi.processor.io.InputStreamCallback;
 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 class FlowFileEvent implements Event {
         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;

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/util/FlowFileEventConstants.java
----------------------------------------------------------------------
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 c9650c1..2c0dd9c 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";
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ControllerServiceInitializer.java
----------------------------------------------------------------------
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 4d1651e..a6c397a 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 class ControllerServiceInitializer implements ConfigurableComponentInitia
 
             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);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ProcessorInitializer.java
----------------------------------------------------------------------
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 7a66f72..745c27c 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 class ProcessorInitializer implements ConfigurableComponentInitializer {
 
             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);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/961be21a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ReportingTaskingInitializer.java
----------------------------------------------------------------------
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 32e878c..3cf86c9 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 class ReportingTaskingInitializer implements ConfigurableComponentInitial
         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);
         }
     }
 }


[4/6] nifi git commit: NIFI-1157 searched for and resolved all remaining references to deprecated items that were clearly addressable.

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/UnpackContent.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/UnpackContent.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/UnpackContent.java
index 6f9e7a2..36cd4ad 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/UnpackContent.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/UnpackContent.java
@@ -30,6 +30,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
+import java.util.concurrent.atomic.AtomicReference;
 import java.util.regex.Pattern;
 
 import org.apache.commons.compress.archivers.ArchiveEntry;
@@ -69,7 +70,6 @@ import org.apache.nifi.util.FlowFileUnpackager;
 import org.apache.nifi.util.FlowFileUnpackagerV1;
 import org.apache.nifi.util.FlowFileUnpackagerV2;
 import org.apache.nifi.util.FlowFileUnpackagerV3;
-import org.apache.nifi.util.ObjectHolder;
 
 @EventDriven
 @SideEffectFree
@@ -415,7 +415,7 @@ public class UnpackContent extends AbstractProcessor {
                 public void process(final InputStream rawIn) throws IOException {
                     try (final InputStream in = new BufferedInputStream(rawIn)) {
                         while (unpackager.hasMoreData()) {
-                            final ObjectHolder<Map<String, String>> attributesRef = new ObjectHolder<>(null);
+                            final AtomicReference<Map<String, String>> attributesRef = new AtomicReference<>(null);
                             FlowFile unpackedFile = session.create(source);
                             try {
                                 unpackedFile = session.write(unpackedFile, new OutputStreamCallback() {

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateXml.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateXml.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateXml.java
index 4647028..d9f9b50 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateXml.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateXml.java
@@ -24,6 +24,7 @@ import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 
 import javax.xml.transform.stream.StreamSource;
@@ -49,7 +50,6 @@ import org.apache.nifi.processor.ProcessorInitializationContext;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.io.InputStreamCallback;
 import org.apache.nifi.processor.util.StandardValidators;
-import org.apache.nifi.util.BooleanHolder;
 import org.xml.sax.SAXException;
 
 @EventDriven
@@ -128,7 +128,7 @@ public class ValidateXml extends AbstractProcessor {
         final ComponentLog logger = getLogger();
 
         for (final FlowFile flowFile : flowFiles) {
-            final BooleanHolder valid = new BooleanHolder(true);
+            final AtomicBoolean valid = new AtomicBoolean(true);
             session.read(flowFile, new InputStreamCallback() {
                 @Override
                 public void process(final InputStream in) throws IOException {

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/servlets/ListenHTTPServlet.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/servlets/ListenHTTPServlet.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/servlets/ListenHTTPServlet.java
index 24cfe1d..5f67e31 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/servlets/ListenHTTPServlet.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/servlets/ListenHTTPServlet.java
@@ -230,9 +230,6 @@ public class ListenHTTPServlet extends HttpServlet {
                                     }
                                 }
 
-                                // remove deprecated FlowFile attribute that was used in older versions of NiFi
-                                attributes.remove("parent.uuid");
-
                                 hasMoreData.set(unpackager.hasMoreData());
                             }
                         }

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/Bin.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/Bin.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/Bin.java
deleted file mode 100644
index 09cccbd..0000000
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/Bin.java
+++ /dev/null
@@ -1,170 +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.util;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-import java.util.regex.Pattern;
-
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.processor.ProcessSession;
-
-/**
- * Note: {@code Bin} objects are NOT thread safe. If multiple threads access a {@code Bin}, the caller must synchronize
- * access.
- * @deprecated As of release 0.5.0, replaced by
- * {@link org.apache.nifi.processor.util.bin.Bin}
- */
-@Deprecated
-public class Bin {
-
-    private final long creationMomentEpochNs;
-    private final long minimumSizeBytes;
-    private final long maximumSizeBytes;
-
-    private volatile int minimumEntries = 0;
-    private volatile int maximumEntries = Integer.MAX_VALUE;
-    private final String fileCountAttribute;
-
-    final List<FlowFileSessionWrapper> binContents = new ArrayList<>();
-    long size;
-    int successiveFailedOfferings = 0;
-
-    /**
-     * Constructs a new bin
-     *
-     * @param minSizeBytes min bytes
-     * @param maxSizeBytes max bytes
-     * @param minEntries min entries
-     * @param maxEntries max entries
-     * @param fileCountAttribute num files
-     * @throws IllegalArgumentException if the min is not less than or equal to the max.
-     */
-    public Bin(final long minSizeBytes, final long maxSizeBytes, final int minEntries, final int maxEntries, final String fileCountAttribute) {
-        this.minimumSizeBytes = minSizeBytes;
-        this.maximumSizeBytes = maxSizeBytes;
-        this.minimumEntries = minEntries;
-        this.maximumEntries = maxEntries;
-        this.fileCountAttribute = fileCountAttribute;
-
-        this.creationMomentEpochNs = System.nanoTime();
-        if (minSizeBytes > maxSizeBytes) {
-            throw new IllegalArgumentException();
-        }
-    }
-
-    /**
-     * Indicates whether the bin has enough items to be considered full. This is based on whether the current size of the bin is greater than the minimum size in bytes and based on having a number of
-     * successive unsuccessful attempts to add a new item (because it is so close to the max or the size of the objects being attempted do not favor tight packing)
-     *
-     * @return true if considered full; false otherwise
-     */
-    public boolean isFull() {
-        return (((size >= minimumSizeBytes) && binContents.size() >= minimumEntries) && (successiveFailedOfferings > 5))
-                || (size >= maximumSizeBytes) || (binContents.size() >= maximumEntries);
-    }
-
-    /**
-     * Indicates enough size exists to meet the minimum requirements
-     *
-     * @return true if full enough
-     */
-    public boolean isFullEnough() {
-        return isFull() || (size >= minimumSizeBytes && (binContents.size() >= minimumEntries));
-    }
-
-    /**
-     * Determines if this bin is older than the time specified.
-     *
-     * @param duration duration
-     * @param unit unit
-     * @return true if this bin is older than the length of time given; false otherwise
-     */
-    public boolean isOlderThan(final int duration, final TimeUnit unit) {
-        final long ageInNanos = System.nanoTime() - creationMomentEpochNs;
-        return ageInNanos > TimeUnit.NANOSECONDS.convert(duration, unit);
-    }
-
-    /**
-     * Determines if this bin is older than the specified bin
-     *
-     * @param other other bin
-     * @return true if this is older than given bin
-     */
-    public boolean isOlderThan(final Bin other) {
-        return creationMomentEpochNs < other.creationMomentEpochNs;
-    }
-
-    /**
-     * If this bin has enough room for the size of the given flow file then it is added otherwise it is not
-     *
-     * @param flowFile flowfile to offer
-     * @param session the ProcessSession to which the FlowFile belongs
-     * @return true if added; false otherwise
-     */
-    public boolean offer(final FlowFile flowFile, final ProcessSession session) {
-        if (((size + flowFile.getSize()) > maximumSizeBytes) || (binContents.size() >= maximumEntries)) {
-            successiveFailedOfferings++;
-            return false;
-        }
-
-        if (fileCountAttribute != null) {
-            final String countValue = flowFile.getAttribute(fileCountAttribute);
-            final Integer count = toInteger(countValue);
-            if (count != null) {
-                final int currentMaximumEntries = this.maximumEntries;
-                this.maximumEntries = Math.min(count, currentMaximumEntries);
-                this.minimumEntries = currentMaximumEntries;
-            }
-        }
-
-        size += flowFile.getSize();
-        binContents.add(new FlowFileSessionWrapper(flowFile, session));
-        successiveFailedOfferings = 0;
-        return true;
-    }
-
-    private static final Pattern intPattern = Pattern.compile("\\d+");
-
-    public Integer toInteger(final String value) {
-        if (value == null) {
-            return null;
-        }
-        if (!intPattern.matcher(value).matches()) {
-            return null;
-        }
-
-        try {
-            return Integer.parseInt(value);
-        } catch (final Exception e) {
-            return null;
-        }
-    }
-
-    /**
-     * @return the underlying list of flow files within this bin
-     */
-    public List<FlowFileSessionWrapper> getContents() {
-        return binContents;
-    }
-
-    public long getBinAge() {
-        final long ageInNanos = System.nanoTime() - creationMomentEpochNs;
-        return TimeUnit.MILLISECONDS.convert(ageInNanos, TimeUnit.NANOSECONDS);
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/BinManager.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/BinManager.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/BinManager.java
deleted file mode 100644
index 3867114..0000000
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/BinManager.java
+++ /dev/null
@@ -1,238 +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.util;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.processor.ProcessSession;
-
-/**
- * This class is thread safe
- *
- * @deprecated As of release 0.5.0, replaced by
- * {@link org.apache.nifi.processor.util.bin.BinManager}
- */
-@Deprecated
-public class BinManager {
-
-    private final AtomicLong minSizeBytes = new AtomicLong(0L);
-    private final AtomicLong maxSizeBytes = new AtomicLong(Long.MAX_VALUE);
-    private final AtomicInteger minEntries = new AtomicInteger(0);
-    private final AtomicInteger maxEntries = new AtomicInteger(Integer.MAX_VALUE);
-    private final AtomicReference<String> fileCountAttribute = new AtomicReference<>(null);
-
-    private final AtomicInteger maxBinAgeSeconds = new AtomicInteger(Integer.MAX_VALUE);
-    private final Map<String, List<Bin>> groupBinMap = new HashMap<>();
-    private final ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock();
-    private final Lock rLock = rwLock.readLock();
-    private final Lock wLock = rwLock.writeLock();
-
-    private int binCount = 0;   // guarded by read/write lock
-
-    public BinManager() {
-    }
-
-    public void purge() {
-        wLock.lock();
-        try {
-            for (final List<Bin> binList : groupBinMap.values()) {
-                for (final Bin bin : binList) {
-                    for (final FlowFileSessionWrapper wrapper : bin.getContents()) {
-                        wrapper.getSession().rollback();
-                    }
-                }
-            }
-            groupBinMap.clear();
-            binCount = 0;
-        } finally {
-            wLock.unlock();
-        }
-    }
-
-    public void setFileCountAttribute(final String fileCountAttribute) {
-        this.fileCountAttribute.set(fileCountAttribute);
-    }
-
-    public void setMinimumEntries(final int minimumEntries) {
-        this.minEntries.set(minimumEntries);
-    }
-
-    public void setMaximumEntries(final int maximumEntries) {
-        this.maxEntries.set(maximumEntries);
-    }
-
-    public int getBinCount() {
-        rLock.lock();
-        try {
-            return binCount;
-        } finally {
-            rLock.unlock();
-        }
-    }
-
-    public void setMinimumSize(final long numBytes) {
-        minSizeBytes.set(numBytes);
-    }
-
-    public void setMaximumSize(final long numBytes) {
-        maxSizeBytes.set(numBytes);
-    }
-
-    public void setMaxBinAge(final int seconds) {
-        maxBinAgeSeconds.set(seconds);
-    }
-
-    /**
-     * Adds the given flowFile to the first available bin in which it fits for the given group or creates a new bin in the specified group if necessary.
-     * <p/>
-     * @param groupIdentifier the group to which the flow file belongs; can be null
-     * @param flowFile the flow file to bin
-     * @param session the ProcessSession to which the FlowFile belongs
-     * @return true if added; false if no bin exists which can fit this item and no bin can be created based on current min/max criteria
-     */
-    public boolean offer(final String groupIdentifier, final FlowFile flowFile, final ProcessSession session) {
-        final long currentMaxSizeBytes = maxSizeBytes.get();
-        if (flowFile.getSize() > currentMaxSizeBytes) { //won't fit into any new bins (and probably none existing)
-            return false;
-        }
-        wLock.lock();
-        try {
-            final List<Bin> currentBins = groupBinMap.get(groupIdentifier);
-            if (currentBins == null) { // this is a new group we need to register
-                final List<Bin> bins = new ArrayList<>();
-                final Bin bin = new Bin(minSizeBytes.get(), currentMaxSizeBytes, minEntries.get(), maxEntries.get(), fileCountAttribute.get());
-                bins.add(bin);
-                groupBinMap.put(groupIdentifier, bins);
-                binCount++;
-                return bin.offer(flowFile, session);
-            } else {
-                for (final Bin bin : currentBins) {
-                    final boolean accepted = bin.offer(flowFile, session);
-                    if (accepted) {
-                        return true;
-                    }
-                }
-
-                //if we've reached this point then we couldn't fit it into any existing bins - gotta make a new one
-                final Bin bin = new Bin(minSizeBytes.get(), currentMaxSizeBytes, minEntries.get(), maxEntries.get(), fileCountAttribute.get());
-                currentBins.add(bin);
-                binCount++;
-                return bin.offer(flowFile, session);
-            }
-        } finally {
-            wLock.unlock();
-        }
-    }
-
-    /**
-     * Finds all bins that are considered full and removes them from the manager.
-     * <p/>
-     * @param relaxFullnessConstraint if false will require bins to be full before considered ready; if true bins only have to meet their minimum size criteria or be 'old' and then they'll be
-     * considered ready
-     * @return bins that are considered full
-     */
-    public Collection<Bin> removeReadyBins(boolean relaxFullnessConstraint) {
-        final Map<String, List<Bin>> newGroupMap = new HashMap<>();
-        final List<Bin> readyBins = new ArrayList<>();
-
-        wLock.lock();
-        try {
-            for (final Map.Entry<String, List<Bin>> group : groupBinMap.entrySet()) {
-                final List<Bin> remainingBins = new ArrayList<>();
-                for (final Bin bin : group.getValue()) {
-                    if (relaxFullnessConstraint && (bin.isFullEnough() || bin.isOlderThan(maxBinAgeSeconds.get(), TimeUnit.SECONDS))) { //relaxed check
-                        readyBins.add(bin);
-                    } else if (!relaxFullnessConstraint && bin.isFull()) { //strict check
-                        readyBins.add(bin);
-                    } else { //it isn't time yet...
-                        remainingBins.add(bin);
-                    }
-                }
-                if (!remainingBins.isEmpty()) {
-                    newGroupMap.put(group.getKey(), remainingBins);
-                }
-            }
-            groupBinMap.clear();
-            groupBinMap.putAll(newGroupMap);
-            binCount -= readyBins.size();
-        } finally {
-            wLock.unlock();
-        }
-        return readyBins;
-    }
-
-    public Bin removeOldestBin() {
-        wLock.lock();
-        try {
-            Bin oldestBin = null;
-            String oldestBinGroup = null;
-
-            for (final Map.Entry<String, List<Bin>> group : groupBinMap.entrySet()) {
-                for (final Bin bin : group.getValue()) {
-                    if (oldestBin == null || bin.isOlderThan(oldestBin)) {
-                        oldestBin = bin;
-                        oldestBinGroup = group.getKey();
-                    }
-                }
-            }
-
-            if (oldestBin == null) {
-                return null;
-            }
-
-            binCount--;
-            final List<Bin> bins = groupBinMap.get(oldestBinGroup);
-            bins.remove(oldestBin);
-            if (bins.isEmpty()) {
-                groupBinMap.remove(oldestBinGroup);
-            }
-            return oldestBin;
-        } finally {
-            wLock.unlock();
-        }
-    }
-
-    /**
-     * @return true if any current bins are older than the allowable max
-     */
-    public boolean containsOldBins() {
-        rLock.lock();
-        try {
-            for (final List<Bin> bins : groupBinMap.values()) {
-                for (final Bin bin : bins) {
-                    if (bin.isOlderThan(maxBinAgeSeconds.get(), TimeUnit.SECONDS)) {
-                        return true;
-                    }
-                }
-            }
-        } finally {
-            rLock.unlock();
-        }
-        return false;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FlowFileSessionWrapper.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FlowFileSessionWrapper.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FlowFileSessionWrapper.java
deleted file mode 100644
index 89ec114..0000000
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FlowFileSessionWrapper.java
+++ /dev/null
@@ -1,49 +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.util;
-
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.processor.ProcessSession;
-
-/**
- * @deprecated As of release 0.5.0, replaced by
- * {@link org.apache.nifi.processor.util.FlowFileSessionWrapper}
- */
-@Deprecated
-public class FlowFileSessionWrapper {
-
-    private final FlowFile flowFile;
-    private final ProcessSession session;
-
-    public FlowFileSessionWrapper(final FlowFile flowFile, final ProcessSession session) {
-        this.flowFile = flowFile;
-        this.session = session;
-    }
-
-    public FlowFile getFlowFile() {
-        return flowFile;
-    }
-
-    public ProcessSession getSession() {
-        return session;
-    }
-
-    @Override
-    public String toString() {
-        return flowFile.toString();
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/BcryptCipherProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/BcryptCipherProvider.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/BcryptCipherProvider.java
index 8c5f464..125aea7 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/BcryptCipherProvider.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/BcryptCipherProvider.java
@@ -95,27 +95,6 @@ public class BcryptCipherProvider extends RandomIVPBECipherProvider {
 
     /**
      * Returns an initialized cipher for the specified algorithm. The key (and IV if necessary) are derived by the KDF of the implementation.
-     * <p>
-     * This method is deprecated because while Bcrypt could generate a random salt to use, it would not be returned to the caller of this method and future derivations would fail. Provide a valid
-     * salt generated by {@link BcryptCipherProvider#generateSalt()}.
-     * </p>
-     *
-     * @param encryptionMethod the {@link EncryptionMethod}
-     * @param password         the secret input
-     * @param keyLength        the desired key length in bits
-     * @param encryptMode      true for encrypt, false for decrypt
-     * @return the initialized cipher
-     * @throws Exception if there is a problem initializing the cipher
-     * @deprecated Provide a salt parameter using {@link BcryptCipherProvider#getCipher(EncryptionMethod, String, byte[], int, boolean)}
-     */
-    @Deprecated
-    @Override
-    public Cipher getCipher(EncryptionMethod encryptionMethod, String password, int keyLength, boolean encryptMode) throws Exception {
-        throw new UnsupportedOperationException("The cipher cannot be initialized without a valid salt. Use BcryptCipherProvider#generateSalt() to generate a valid salt");
-    }
-
-    /**
-     * Returns an initialized cipher for the specified algorithm. The key (and IV if necessary) are derived by the KDF of the implementation.
      *
      * The IV can be retrieved by the calling method using {@link Cipher#getIV()}.
      *

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/NiFiLegacyCipherProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/NiFiLegacyCipherProvider.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/NiFiLegacyCipherProvider.java
index 17295d7..c462629 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/NiFiLegacyCipherProvider.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/NiFiLegacyCipherProvider.java
@@ -46,22 +46,6 @@ public class NiFiLegacyCipherProvider extends OpenSSLPKCS5CipherProvider impleme
 
     /**
      * Returns an initialized cipher for the specified algorithm. The key (and IV if necessary) are derived using the NiFi legacy code, based on @see org.apache.nifi.processors.standard.util.crypto
-     * .OpenSSLPKCS5CipherProvider#getCipher(java.lang.String, java.lang.String, java.lang.String, boolean) [essentially {@code MD5(password || salt) * 1000 }].
-     *
-     * @param encryptionMethod the {@link EncryptionMethod}
-     * @param password         the secret input
-     * @param keyLength        the desired key length in bits (ignored because OpenSSL ciphers provide key length in algorithm name)
-     * @param encryptMode      true for encrypt, false for decrypt
-     * @return the initialized cipher
-     * @throws Exception if there is a problem initializing the cipher
-     */
-    @Override
-    public Cipher getCipher(EncryptionMethod encryptionMethod, String password, int keyLength, boolean encryptMode) throws Exception {
-        return getCipher(encryptionMethod, password, new byte[0], keyLength, encryptMode);
-    }
-
-    /**
-     * Returns an initialized cipher for the specified algorithm. The key (and IV if necessary) are derived using the NiFi legacy code, based on @see org.apache.nifi.processors.standard.util.crypto
      * .OpenSSLPKCS5CipherProvider#getCipher(java.lang.String, java.lang.String, java.lang.String, byte[], boolean) [essentially {@code MD5(password || salt) * 1000 }].
      *
      * @param encryptionMethod the {@link EncryptionMethod}
@@ -145,6 +129,7 @@ public class NiFiLegacyCipherProvider extends OpenSSLPKCS5CipherProvider impleme
         out.write(salt);
     }
 
+    @Override
     protected int getIterationCount() {
         return ITERATION_COUNT;
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/OpenSSLPKCS5CipherProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/OpenSSLPKCS5CipherProvider.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/OpenSSLPKCS5CipherProvider.java
index a18d4fd..4253f9b 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/OpenSSLPKCS5CipherProvider.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/OpenSSLPKCS5CipherProvider.java
@@ -58,22 +58,6 @@ public class OpenSSLPKCS5CipherProvider implements PBECipherProvider {
      *
      * @param encryptionMethod the {@link EncryptionMethod}
      * @param password         the secret input
-     * @param keyLength        the desired key length in bits (ignored because OpenSSL ciphers provide key length in algorithm name)
-     * @param encryptMode      true for encrypt, false for decrypt
-     * @return the initialized cipher
-     * @throws Exception if there is a problem initializing the cipher
-     */
-    @Override
-    public Cipher getCipher(EncryptionMethod encryptionMethod, String password, int keyLength, boolean encryptMode) throws Exception {
-        return getCipher(encryptionMethod, password, new byte[0], keyLength, encryptMode);
-    }
-
-    /**
-     * Returns an initialized cipher for the specified algorithm. The key (and IV if necessary) are derived using the
-     * <a href="https://www.openssl.org/docs/manmaster/crypto/EVP_BytesToKey.html">OpenSSL EVP_BytesToKey proprietary KDF</a> [essentially {@code MD5(password || salt) }].
-     *
-     * @param encryptionMethod the {@link EncryptionMethod}
-     * @param password         the secret input
      * @param salt             the salt
      * @param keyLength        the desired key length in bits (ignored because OpenSSL ciphers provide key length in algorithm name)
      * @param encryptMode      true for encrypt, false for decrypt

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/PBECipherProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/PBECipherProvider.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/PBECipherProvider.java
index 8677656..4d9fcfc 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/PBECipherProvider.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/PBECipherProvider.java
@@ -24,17 +24,6 @@ import java.io.InputStream;
 import java.io.OutputStream;
 
 public interface PBECipherProvider extends CipherProvider {
-    /**
-     * Returns an initialized cipher for the specified algorithm. The key (and IV if necessary) are derived by the KDF of the implementation.
-     *
-     * @param encryptionMethod the {@link EncryptionMethod}
-     * @param password         the secret input
-     * @param keyLength        the desired key length in bits
-     * @param encryptMode      true for encrypt, false for decrypt
-     * @return the initialized cipher
-     * @throws Exception if there is a problem initializing the cipher
-     */
-    Cipher getCipher(EncryptionMethod encryptionMethod, String password, int keyLength, boolean encryptMode) throws Exception;
 
     /**
      * Returns an initialized cipher for the specified algorithm. The key (and IV if necessary) are derived by the KDF of the implementation.

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/PBKDF2CipherProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/PBKDF2CipherProvider.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/PBKDF2CipherProvider.java
index 748d77f..ee8d5d3 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/PBKDF2CipherProvider.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/PBKDF2CipherProvider.java
@@ -108,23 +108,6 @@ public class PBKDF2CipherProvider extends RandomIVPBECipherProvider {
      *
      * @param encryptionMethod the {@link EncryptionMethod}
      * @param password         the secret input
-     * @param keyLength        the desired key length in bits
-     * @param encryptMode      true for encrypt, false for decrypt
-     * @return the initialized cipher
-     * @throws Exception if there is a problem initializing the cipher
-     */
-    @Override
-    public Cipher getCipher(EncryptionMethod encryptionMethod, String password, int keyLength, boolean encryptMode) throws Exception {
-        return getCipher(encryptionMethod, password, new byte[0], new byte[0], keyLength, encryptMode);
-    }
-
-    /**
-     * Returns an initialized cipher for the specified algorithm. The key (and IV if necessary) are derived by the KDF of the implementation.
-     *
-     * The IV can be retrieved by the calling method using {@link Cipher#getIV()}.
-     *
-     * @param encryptionMethod the {@link EncryptionMethod}
-     * @param password         the secret input
      * @param salt             the salt
      * @param keyLength        the desired key length in bits
      * @param encryptMode      true for encrypt, false for decrypt

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/ScryptCipherProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/ScryptCipherProvider.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/ScryptCipherProvider.java
index 635b4ef..15a29e2 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/ScryptCipherProvider.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/crypto/ScryptCipherProvider.java
@@ -113,26 +113,6 @@ public class ScryptCipherProvider extends RandomIVPBECipherProvider {
 
     /**
      * Returns an initialized cipher for the specified algorithm. The key (and IV if necessary) are derived by the KDF of the implementation.
-     * <p/>
-     * This method is deprecated because while Scrypt could generate a random salt to use, it would not be returned to the caller of this method and future derivations would fail. Provide a valid
-     * salt generated by {@link ScryptCipherProvider#generateSalt()}.
-     *
-     * @param encryptionMethod the {@link EncryptionMethod}
-     * @param password         the secret input
-     * @param keyLength        the desired key length in bits
-     * @param encryptMode      true for encrypt, false for decrypt
-     * @return the initialized cipher
-     * @throws Exception if there is a problem initializing the cipher
-     * @deprecated Provide a salt parameter using {@link ScryptCipherProvider#getCipher(EncryptionMethod, String, byte[], int, boolean)}
-     */
-    @Deprecated
-    @Override
-    public Cipher getCipher(EncryptionMethod encryptionMethod, String password, int keyLength, boolean encryptMode) throws Exception {
-        throw new UnsupportedOperationException("The cipher cannot be initialized without a valid salt. Use ScryptCipherProvider#generateSalt() to generate a valid salt");
-    }
-
-    /**
-     * Returns an initialized cipher for the specified algorithm. The key (and IV if necessary) are derived by the KDF of the implementation.
      *
      * The IV can be retrieved by the calling method using {@link Cipher#getIV()}.
      *

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/groovy/org/apache/nifi/processors/standard/util/crypto/BcryptCipherProviderGroovyTest.groovy
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/groovy/org/apache/nifi/processors/standard/util/crypto/BcryptCipherProviderGroovyTest.groovy b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/groovy/org/apache/nifi/processors/standard/util/crypto/BcryptCipherProviderGroovyTest.groovy
index 84b91c6..396e3b2 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/groovy/org/apache/nifi/processors/standard/util/crypto/BcryptCipherProviderGroovyTest.groovy
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/groovy/org/apache/nifi/processors/standard/util/crypto/BcryptCipherProviderGroovyTest.groovy
@@ -31,7 +31,6 @@ import org.junit.runner.RunWith
 import org.junit.runners.JUnit4
 import org.slf4j.Logger
 
-//import org.mindrot.jbcrypt.BCrypt
 import org.slf4j.LoggerFactory
 
 import javax.crypto.Cipher
@@ -345,16 +344,7 @@ public class BcryptCipherProviderGroovyTest {
         // Two different errors -- one explaining the no-salt method is not supported, and the other for an empty byte[] passed
 
         // Act
-        def msg = shouldFail(UnsupportedOperationException) {
-            Cipher cipher = cipherProvider.getCipher(encryptionMethod, PASSWORD, DEFAULT_KEY_LENGTH, true);
-        }
-        logger.expected(msg)
-
-        // Assert
-        assert msg =~ "The cipher cannot be initialized without a valid salt\\. Use BcryptCipherProvider#generateSalt\\(\\) to generate a valid salt"
-
-        // Act
-        msg = shouldFail(IllegalArgumentException) {
+        def msg = shouldFail(IllegalArgumentException) {
             Cipher cipher = cipherProvider.getCipher(encryptionMethod, PASSWORD, new byte[0], DEFAULT_KEY_LENGTH, true);
         }
         logger.expected(msg)

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/groovy/org/apache/nifi/processors/standard/util/crypto/ScryptCipherProviderGroovyTest.groovy
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/groovy/org/apache/nifi/processors/standard/util/crypto/ScryptCipherProviderGroovyTest.groovy b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/groovy/org/apache/nifi/processors/standard/util/crypto/ScryptCipherProviderGroovyTest.groovy
index 08380e5..8fd0455 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/groovy/org/apache/nifi/processors/standard/util/crypto/ScryptCipherProviderGroovyTest.groovy
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/groovy/org/apache/nifi/processors/standard/util/crypto/ScryptCipherProviderGroovyTest.groovy
@@ -349,19 +349,8 @@ public class ScryptCipherProviderGroovyTest {
         EncryptionMethod encryptionMethod = EncryptionMethod.AES_CBC
         logger.info("Using algorithm: ${encryptionMethod.getAlgorithm()}");
 
-        // Two different errors -- one explaining the no-salt method is not supported, and the other for an empty byte[] passed
-
-        // Act
-        def msg = shouldFail(UnsupportedOperationException) {
-            Cipher cipher = cipherProvider.getCipher(encryptionMethod, PASSWORD, DEFAULT_KEY_LENGTH, true);
-        }
-        logger.expected(msg)
-
-        // Assert
-        assert msg =~ "The cipher cannot be initialized without a valid salt\\. Use ScryptCipherProvider#generateSalt\\(\\) to generate a valid salt"
-
         // Act
-        msg = shouldFail(IllegalArgumentException) {
+        def msg = shouldFail(IllegalArgumentException) {
             Cipher cipher = cipherProvider.getCipher(encryptionMethod, PASSWORD, new byte[0], DEFAULT_KEY_LENGTH, true);
         }
         logger.expected(msg)

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestListenSyslog.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestListenSyslog.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestListenSyslog.java
index 4731a10..413ac7c 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestListenSyslog.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestListenSyslog.java
@@ -29,7 +29,6 @@ import org.apache.nifi.processors.standard.syslog.SyslogEvent;
 import org.apache.nifi.processors.standard.syslog.SyslogParser;
 import org.apache.nifi.provenance.ProvenanceEventRecord;
 import org.apache.nifi.provenance.ProvenanceEventType;
-import org.apache.nifi.util.IntegerHolder;
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
@@ -48,6 +47,7 @@ import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import static org.junit.Assert.assertEquals;
 
@@ -399,7 +399,7 @@ public class TestListenSyslog {
 
         // Add message that will throw a FlowFileAccessException the first time that we attempt to read
         // the contents but will succeed the second time.
-        final IntegerHolder getMessageAttempts = new IntegerHolder(0);
+        final AtomicInteger getMessageAttempts = new AtomicInteger(0);
         msgs.add(new ListenSyslog.RawSyslogEvent(VALID_MESSAGE.getBytes(), "sender-01") {
             @Override
             public byte[] getData() {

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DatabaseSystemDescriptor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DatabaseSystemDescriptor.java b/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DatabaseSystemDescriptor.java
deleted file mode 100644
index 00e30ca..0000000
--- a/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DatabaseSystemDescriptor.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.dbcp;
-
-import java.text.MessageFormat;
-
-import org.apache.nifi.components.AllowableValue;
-
-/**
- * An immutable object for holding information about a database system.
- *
- */
-@Deprecated
-public class DatabaseSystemDescriptor extends AllowableValue {
-
-    public final String driverClassName;
-    public final Integer defaultPort;
-    public final String urlTemplate;
-    public final boolean internalDriverJar;
-
-    public DatabaseSystemDescriptor(String value, String description, String driverClassName, Integer defaultPort, String urlTemplate, boolean internalDriverJar) {
-        super(value, value, description);
-
-        if (defaultPort==null)
-            throw new IllegalArgumentException("defaultPort cannot be null");
-
-        this.driverClassName = driverClassName;
-        this.defaultPort = defaultPort;
-        this.urlTemplate = urlTemplate;
-        this.internalDriverJar = internalDriverJar;
-    }
-
-    public String buildUrl(String host, Integer port, String dbname) {
-        return MessageFormat.format(urlTemplate, host, port.toString(), dbname);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DatabaseSystems.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DatabaseSystems.java b/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DatabaseSystems.java
deleted file mode 100644
index 943f6d2..0000000
--- a/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DatabaseSystems.java
+++ /dev/null
@@ -1,83 +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.dbcp;
-
-@Deprecated
-public class DatabaseSystems {
-
-    /**
-     * Currently contain only few known Database systems.
-     * Please help to expand this list.
-     *
-     * Please be ensure that all JDBC drivers are license-compatible with Apache.
-     * http://www.apache.org/legal/resolved.html
-     * If not include them in "JDBC driver jar must be loaded from external location" section
-     * and do not include actual driver in NiFi distribution (don't include driver in pom.xml file)
-     *
-     * {0} host name/ip
-     * {1} port number
-     * {2} database name
-     *
-     * for example url template
-     *   "jdbc:postgresql://{0}:{1}/{2}"
-     * will be after building
-     *  "jdbc:postgresql://bighost:5432/Trove"
-     */
-    public static DatabaseSystemDescriptor[] knownDatabaseSystems = {
-        // =================  JDBC driver jar should be included in nar (in pom.xml dependencies) =======================
-
-        new DatabaseSystemDescriptor("Postgres", "PostgreSQL open soure object-relational database.",
-            "org.postgresql.Driver", 5432, "jdbc:postgresql://{0}:{1}/{2}", true),
-
-            new DatabaseSystemDescriptor("JavaDB", "Java DB is Oracle's supported distribution of the Apache Derby open source database. Included in JDK.",
-                "org.apache.derby.jdbc.EmbeddedDriver", 1, "jdbc:derby:{2};create=true", true),
-
-                new DatabaseSystemDescriptor("Derby", "Apache Derby is an open source relational database.",
-                    "org.apache.derby.jdbc.EmbeddedDriver", 1, "jdbc:derby:{2};create=true", true),
-
-
-                    // =================  JDBC driver jar must be loaded from external location  =======================
-                    // Such drivers cannot be included in NiFi distribution because are not license-compatible with Apache.
-                    new DatabaseSystemDescriptor("MariaDB",
-                        "MariaDB is a community-developed fork of the MySQL relational database management system intended to remain free under the GNU GPL.",
-                        "org.mariadb.jdbc.Driver", 3306, "jdbc:mariadb://{0}:{1}/{2}", false),
-
-                        new DatabaseSystemDescriptor("Oracle",
-                            "Oracle Database is an object-relational database management system.",
-                            "oracle.jdbc.OracleDriver", 1521, "jdbc:oracle:thin:@//{0}:{1}/{2}", false),
-
-                            new DatabaseSystemDescriptor("Sybase",
-                                "Sybase is an relational database management system.",
-                                "com.sybase.jdbc3.jdbc.SybDriver", 5000, "jdbc:sybase:Tds:{0}:{1}/{2}", false),
-
-
-                                // =================  Unknown JDBC driver, user must provide connection details =====================
-                                new DatabaseSystemDescriptor("Other DB", "Other JDBC compliant JDBC driver",
-                                    null, 1, null, false),
-
-    };
-
-    public static DatabaseSystemDescriptor getDescriptor(String name) {
-        for ( DatabaseSystemDescriptor descr : DatabaseSystems.knownDatabaseSystems) {
-            if (descr.getValue().equalsIgnoreCase(name))
-                return descr;
-        }
-        throw new IllegalArgumentException("Can't find DatabaseSystemDescriptor by name " + name);
-    }
-
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/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-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/test/java/org/apache/nifi/dbcp/TestDatabaseSystems.java b/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/test/java/org/apache/nifi/dbcp/TestDatabaseSystems.java
deleted file mode 100644
index 3c9064e..0000000
--- a/nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/test/java/org/apache/nifi/dbcp/TestDatabaseSystems.java
+++ /dev/null
@@ -1,31 +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.dbcp;
-
-import static org.apache.nifi.dbcp.DatabaseSystems.getDescriptor;
-import static org.junit.Assert.assertEquals;
-
-import org.junit.Test;
-
-public class TestDatabaseSystems {
-
-    @Test
-    public void testKnownDatabaseSystems() {
-        assertEquals("jdbc:postgresql://bighost:5432/Trove", getDescriptor("Postgres").buildUrl("bighost",5432,"Trove"));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/f987b216/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-service/src/test/java/org/apache/nifi/ssl/TestProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-service/src/test/java/org/apache/nifi/ssl/TestProcessor.java b/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-service/src/test/java/org/apache/nifi/ssl/TestProcessor.java
index 6cb2fb6..440e53c 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-service/src/test/java/org/apache/nifi/ssl/TestProcessor.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-service/src/test/java/org/apache/nifi/ssl/TestProcessor.java
@@ -16,7 +16,6 @@
  */
 package org.apache.nifi.ssl;
 
-import org.apache.nifi.ssl.SSLContextService;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -25,7 +24,6 @@ import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.exception.ProcessException;
-import org.apache.nifi.processor.util.StandardValidators;
 
 public class TestProcessor extends AbstractProcessor {
 
@@ -39,7 +37,7 @@ public class TestProcessor extends AbstractProcessor {
         propDescs.add(new PropertyDescriptor.Builder()
                 .name("SSL Context Svc ID")
                 .description("ID of SSL Context Svc")
-                .addValidator(StandardValidators.createControllerServiceExistsValidator(SSLContextService.class))
+                .identifiesControllerService(SSLContextService.class)
                 .required(true)
                 .build());
         return propDescs;